From 8d1deeeace5c5489ba7dd0bc7465f07f794a3bae Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 23 May 2020 00:19:57 +0800 Subject: [PATCH 01/75] fail casting from integral to timestamp --- .../spark/sql/catalyst/expressions/Cast.scala | 26 ++++++------------- .../sql/catalyst/expressions/CastSuite.scala | 7 +++++ 2 files changed, 15 insertions(+), 18 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 ef70915a5c96..9ab8dc5e6fbe 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 @@ -59,7 +59,7 @@ object Cast { case (StringType, TimestampType) => true case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true - case (_: NumericType, TimestampType) => true + case (_: FractionalType, TimestampType) => true case (StringType, DateType) => true case (TimestampType, DateType) => true @@ -138,10 +138,8 @@ object Cast { case (_: CalendarIntervalType, StringType) => true case (NullType, _) => true - // Spark supports casting between long and timestamp, please see `longToTimestamp` and - // `timestampToLong` for details. + // spark forbid casting from integral to timestamp, more details in [SPARK-31790] case (TimestampType, LongType) => true - case (LongType, TimestampType) => true case (ArrayType(fromType, fn), ArrayType(toType, tn)) => resolvableNullability(fn, tn) && canUpCast(fromType, toType) @@ -266,7 +264,12 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit TypeCheckResult.TypeCheckSuccess } else { TypeCheckResult.TypeCheckFailure( - s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}") + if (child.dataType.isInstanceOf[IntegralType] && dataType.isInstanceOf[TimestampType]) { + s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + + s"please use function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instand" + } else { + s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}" + }) } } @@ -425,14 +428,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs, zoneId).orNull) case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0) - case LongType => - buildCast[Long](_, l => longToTimestamp(l)) - case IntegerType => - buildCast[Int](_, i => longToTimestamp(i.toLong)) - case ShortType => - buildCast[Short](_, s => longToTimestamp(s.toLong)) - case ByteType => - buildCast[Byte](_, b => longToTimestamp(b.toLong)) case DateType => buildCast[Int](_, d => epochDaysToMicros(d, zoneId)) // TimestampWritable.decimalToTimestamp @@ -453,8 +448,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong } - // converting seconds to us - private[this] def longToTimestamp(t: Long): Long = SECONDS.toMicros(t) // converting us to seconds private[this] def timestampToLong(ts: Long): Long = { Math.floorDiv(ts, MICROS_PER_SECOND) @@ -1229,8 +1222,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit """ case BooleanType => (c, evPrim, evNull) => code"$evPrim = $c ? 1L : 0L;" - case _: IntegralType => - (c, evPrim, evNull) => code"$evPrim = ${longToTimeStampCode(c)};" case DateType => val zoneIdClass = classOf[ZoneId] val zid = JavaCode.global( @@ -1277,7 +1268,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val block = inline"new java.math.BigDecimal($MICROS_PER_SECOND)" code"($d.toBigDecimal().bigDecimal().multiply($block)).longValue()" } - private[this] def longToTimeStampCode(l: ExprValue): Block = code"$l * (long)$MICROS_PER_SECOND" private[this] def timestampToLongCode(ts: ExprValue): Block = code"java.lang.Math.floorDiv($ts, $MICROS_PER_SECOND)" private[this] def timestampToDoubleCode(ts: ExprValue): Block = 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 e5bff7f7af00..89345fb8660c 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 @@ -1311,6 +1311,13 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(negativeTs, LongType), expectedSecs) } } + + test("SPARK-31710:fail casting from integral to timestamp by default") { + assert(!cast(2.toByte, TimestampType).resolved) + assert(!cast(10.toShort, TimestampType).resolved) + assert(!cast(3, TimestampType).resolved) + assert(!cast(10L, TimestampType).resolved) + } } /** From a02c6c77486ef314dcb48132653f84cfb2f64a22 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 25 May 2020 23:22:48 +0800 Subject: [PATCH 02/75] 'fix-testcase-fail' --- .../src/test/resources/sql-functions/sql-expression-schema.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 3570fb61e288..0a7efac25805 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -62,7 +62,7 @@ | org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | SELECT cast('2020-02-03 11:11:00,123' as timestamp) | struct | | org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceil | SELECT ceil(-0.1) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceiling | SELECT ceiling(-0.1) | struct | From e61c484e70f326edfbbbab9ce3ec330535acd28c Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 26 May 2020 00:18:08 +0800 Subject: [PATCH 03/75] 'add-restore-flag' --- .../spark/sql/catalyst/expressions/Cast.scala | 27 ++++++++++++++++--- .../apache/spark/sql/internal/SQLConf.scala | 14 +++++++++- .../sql/catalyst/expressions/CastSuite.scala | 22 ++++++++++++--- .../sql-functions/sql-expression-schema.md | 2 +- 4 files changed, 55 insertions(+), 10 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 9ab8dc5e6fbe..ea66c70d2ffa 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 @@ -59,7 +59,8 @@ object Cast { case (StringType, TimestampType) => true case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true - case (_: FractionalType, TimestampType) => true + case (_: NumericType, TimestampType) => + if (SQLConf.get.allowCastNumericToTimestamp) true else false case (StringType, DateType) => true case (TimestampType, DateType) => true @@ -138,8 +139,10 @@ object Cast { case (_: CalendarIntervalType, StringType) => true case (NullType, _) => true - // spark forbid casting from integral to timestamp, more details in [SPARK-31790] + // Spark supports casting between long and timestamp, please see `longToTimestamp` and + // `timestampToLong` for details. case (TimestampType, LongType) => true + case (LongType, TimestampType) => true case (ArrayType(fromType, fn), ArrayType(toType, tn)) => resolvableNullability(fn, tn) && canUpCast(fromType, toType) @@ -264,9 +267,12 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit TypeCheckResult.TypeCheckSuccess } else { TypeCheckResult.TypeCheckFailure( - if (child.dataType.isInstanceOf[IntegralType] && dataType.isInstanceOf[TimestampType]) { + if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) { s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + - s"please use function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instand" + s",you can enable the casting by setting" + + s"spark.sql.legacy.allowCastNumericToTimestamp =true;" + + s"but we strongly recommand using function" + + s"TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instand" } else { s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}" }) @@ -428,6 +434,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs, zoneId).orNull) case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0) + case LongType => + buildCast[Long](_, l => longToTimestamp(l)) + case IntegerType => + buildCast[Int](_, i => longToTimestamp(i.toLong)) + case ShortType => + buildCast[Short](_, s => longToTimestamp(s.toLong)) + case ByteType => + buildCast[Byte](_, b => longToTimestamp(b.toLong)) case DateType => buildCast[Int](_, d => epochDaysToMicros(d, zoneId)) // TimestampWritable.decimalToTimestamp @@ -448,6 +462,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong } + // converting seconds to us + private[this] def longToTimestamp(t: Long): Long = SECONDS.toMicros(t) // converting us to seconds private[this] def timestampToLong(ts: Long): Long = { Math.floorDiv(ts, MICROS_PER_SECOND) @@ -1222,6 +1238,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit """ case BooleanType => (c, evPrim, evNull) => code"$evPrim = $c ? 1L : 0L;" + case _: IntegralType => + (c, evPrim, evNull) => code"$evPrim = ${longToTimeStampCode(c)};" case DateType => val zoneIdClass = classOf[ZoneId] val zid = JavaCode.global( @@ -1268,6 +1286,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val block = inline"new java.math.BigDecimal($MICROS_PER_SECOND)" code"($d.toBigDecimal().bigDecimal().multiply($block)).longValue()" } + private[this] def longToTimeStampCode(l: ExprValue): Block = code"$l * (long)$MICROS_PER_SECOND" private[this] def timestampToLongCode(ts: ExprValue): Block = code"java.lang.Math.floorDiv($ts, $MICROS_PER_SECOND)" private[this] def timestampToDoubleCode(ts: ExprValue): Block = 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 cf0268773c39..476a7ca6ef09 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 @@ -2586,6 +2586,15 @@ object SQLConf { .checkValue(_ > 0, "The timeout value must be positive") .createWithDefault(10L) + val LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP = + buildConf("spark.sql.legacy.allowCastNumericToTimestamp") + .internal() + .doc("When true, allow cast numeric to timestamp, but for integral numbers," + + "Hive treats it as milliseconds, Spark SQL treats n as seconds") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -2884,7 +2893,6 @@ class SQLConf extends Serializable with Logging { def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } - /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. @@ -3165,6 +3173,10 @@ class SQLConf extends Serializable with Logging { def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) + def allowCastNumericToTimestamp: Boolean = + getConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP) + + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 89345fb8660c..62eddcadb840 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 @@ -1313,10 +1313,24 @@ class CastSuite extends CastSuiteBase { } test("SPARK-31710:fail casting from integral to timestamp by default") { - assert(!cast(2.toByte, TimestampType).resolved) - assert(!cast(10.toShort, TimestampType).resolved) - assert(!cast(3, TimestampType).resolved) - assert(!cast(10L, TimestampType).resolved) + withSQLConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { + assert(!cast(2.toByte, TimestampType).resolved) + assert(!cast(10.toShort, TimestampType).resolved) + assert(!cast(3, TimestampType).resolved) + assert(!cast(10L, TimestampType).resolved) + assert(!cast(Decimal(1.2), TimestampType).resolved) + assert(!cast(1.7f, TimestampType).resolved) + assert(!cast(2.3d, TimestampType).resolved) + } + withSQLConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + assert(cast(2.toByte, TimestampType).resolved) + assert(cast(10.toShort, TimestampType).resolved) + assert(cast(3, TimestampType).resolved) + assert(cast(10L, TimestampType).resolved) + assert(cast(Decimal(1.2), TimestampType).resolved) + assert(cast(1.7f, TimestampType).resolved) + assert(cast(2.3d, TimestampType).resolved) + } } } diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 0a7efac25805..063a5fef6c7e 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -62,7 +62,7 @@ | org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | SELECT cast('2020-02-03 11:11:00,123' as timestamp) | struct | +| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceil | SELECT ceil(-0.1) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceiling | SELECT ceiling(-0.1) | struct | From d99cc301d6d11b415dd472edd68db8575cd70ace Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 26 May 2020 00:20:46 +0800 Subject: [PATCH 04/75] 'fix-space' --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../src/test/resources/sql-functions/sql-expression-schema.md | 2 +- 2 files changed, 2 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 476a7ca6ef09..cba9d4be0474 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 @@ -2893,6 +2893,7 @@ class SQLConf extends Serializable with Logging { def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = { LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY)) } + /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. @@ -3176,7 +3177,6 @@ class SQLConf extends Serializable with Logging { def allowCastNumericToTimestamp: Boolean = getConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 063a5fef6c7e..3570fb61e288 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -62,7 +62,7 @@ | org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceil | SELECT ceil(-0.1) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceiling | SELECT ceiling(-0.1) | struct | From 0a1a6a5bef84259bdf9716837a02d9e64c3e063b Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 27 May 2020 11:41:55 +0800 Subject: [PATCH 05/75] fix code style --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 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 ea66c70d2ffa..a0c8ad1349c4 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 @@ -59,8 +59,7 @@ object Cast { case (StringType, TimestampType) => true case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true - case (_: NumericType, TimestampType) => - if (SQLConf.get.allowCastNumericToTimestamp) true else false + case (_: NumericType, TimestampType) => SQLConf.get.allowCastNumericToTimestamp case (StringType, DateType) => true case (TimestampType, DateType) => true @@ -269,10 +268,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit TypeCheckResult.TypeCheckFailure( if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) { s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + - s",you can enable the casting by setting" + - s"spark.sql.legacy.allowCastNumericToTimestamp =true;" + - s"but we strongly recommand using function" + - s"TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instand" + ", you can enable the casting by setting " + + s"${SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + + "but we strongly recommand using function " + + "TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead." } else { s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}" }) From bae99b1648f09a1da90f09e67ba112302dfe85d2 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 27 May 2020 15:07:26 +0800 Subject: [PATCH 06/75] fix code style --- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 27 +++++++------------ 2 files changed, 11 insertions(+), 18 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 a0c8ad1349c4..57c69276e9c7 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 @@ -268,7 +268,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit TypeCheckResult.TypeCheckFailure( if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) { s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + - ", you can enable the casting by setting " + + "you can enable the casting by setting " + s"${SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + "but we strongly recommand using function " + "TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead." 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 62eddcadb840..6c26709e015d 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 @@ -1313,23 +1313,16 @@ class CastSuite extends CastSuiteBase { } test("SPARK-31710:fail casting from integral to timestamp by default") { - withSQLConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { - assert(!cast(2.toByte, TimestampType).resolved) - assert(!cast(10.toShort, TimestampType).resolved) - assert(!cast(3, TimestampType).resolved) - assert(!cast(10L, TimestampType).resolved) - assert(!cast(Decimal(1.2), TimestampType).resolved) - assert(!cast(1.7f, TimestampType).resolved) - assert(!cast(2.3d, TimestampType).resolved) - } - withSQLConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - assert(cast(2.toByte, TimestampType).resolved) - assert(cast(10.toShort, TimestampType).resolved) - assert(cast(3, TimestampType).resolved) - assert(cast(10L, TimestampType).resolved) - assert(cast(Decimal(1.2), TimestampType).resolved) - assert(cast(1.7f, TimestampType).resolved) - assert(cast(2.3d, TimestampType).resolved) + Seq(true, false).foreach { enable => + withSQLConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { + assert(cast(2.toByte, TimestampType).resolved == enable) + assert(cast(10.toShort, TimestampType).resolved == enable) + assert(cast(3, TimestampType).resolved == enable) + assert(cast(10L, TimestampType).resolved == enable) + assert(cast(Decimal(1.2), TimestampType).resolved == enable) + assert(cast(1.7f, TimestampType).resolved == enable) + assert(cast(2.3d, TimestampType).resolved == enable) + } } } } From 39da7bebe7123acbedea697a2e88bd85ccc4bcef Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 27 May 2020 17:05:50 +0800 Subject: [PATCH 07/75] improve code --- docs/sql-migration-guide.md | 2 ++ .../scala/org/apache/spark/sql/catalyst/expressions/Cast.scala | 3 ++- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 -- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 96f2c5dcf973..dcf9b9f1e7c8 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -115,6 +115,8 @@ license: | - In Spark 3.0, when casting interval values to string type, there is no "interval" prefix, for example, `1 days 2 hours`. In Spark version 2.4 and below, the string contains the "interval" prefix like `interval 1 days 2 hours`. - In 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, for example, `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 below, when casting string to integrals and booleans, it does not trim the whitespaces from both ends; the foregoing results is `null`, while to datetimes, only the trailing spaces (= ASCII 32) are removed. + + - In Spark 3.0, casting numeric to timestamp will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. ### Query Engine 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 57c69276e9c7..5ff5bf38804e 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 @@ -59,7 +59,8 @@ object Cast { case (StringType, TimestampType) => true case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true - case (_: NumericType, TimestampType) => SQLConf.get.allowCastNumericToTimestamp + case (_: NumericType, TimestampType) => + SQLConf.get.getConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP) case (StringType, DateType) => true case (TimestampType, DateType) => 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 cba9d4be0474..dd74dccfc9bf 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 @@ -3174,8 +3174,6 @@ class SQLConf extends Serializable with Logging { def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) - def allowCastNumericToTimestamp: Boolean = - getConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP) /** ********************** SQLConf functionality methods ************ */ From 0189b91a4683a69cf5f34de38b3e417820b6c7fb Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 28 May 2020 10:15:25 +0800 Subject: [PATCH 08/75] fix test cases failure --- docs/sql-migration-guide.md | 4 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 133 +++++++++++++----- 4 files changed, 100 insertions(+), 43 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index dcf9b9f1e7c8..8d11e6611e5f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -27,6 +27,8 @@ license: | - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. + + - In Spark 3.1, casting numeric to timestamp will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. ## Upgrading from Spark SQL 2.4 to 3.0 @@ -116,8 +118,6 @@ license: | - In 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, for example, `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 below, when casting string to integrals and booleans, it does not trim the whitespaces from both ends; the foregoing results is `null`, while to datetimes, only the trailing spaces (= ASCII 32) are removed. - - In Spark 3.0, casting numeric to timestamp will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. - ### Query Engine - In Spark version 2.4 and below, 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. These queries are treated as invalid in Spark 3.0. 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 5ff5bf38804e..11d8c413070a 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 @@ -60,7 +60,7 @@ object Cast { case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true case (_: NumericType, TimestampType) => - SQLConf.get.getConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP) + SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) case (StringType, DateType) => true case (TimestampType, DateType) => true @@ -270,7 +270,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) { s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + "you can enable the casting by setting " + - s"${SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + + s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + "but we strongly recommand using function " + "TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead." } else { 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 dd74dccfc9bf..a575f70da6e1 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 @@ -2586,7 +2586,7 @@ object SQLConf { .checkValue(_ > 0, "The timeout value must be positive") .createWithDefault(10L) - val LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP = + val LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP = buildConf("spark.sql.legacy.allowCastNumericToTimestamp") .internal() .doc("When true, allow cast numeric to timestamp, but for integral numbers," + 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 6c26709e015d..8a5697a50887 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 @@ -50,7 +50,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } protected def checkNullCast(from: DataType, to: DataType): Unit = { - checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + } + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { + if (from.isInstanceOf[NumericType] && to.isInstanceOf[TimestampType]) { + assert(!cast(Literal.create(null, from), to, UTC_OPT).resolved) + } else { + checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + } + } } test("null cast") { @@ -239,8 +249,13 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast(1.5, 1.5f) checkCast(1.5, "1.5") - checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) - checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + } + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { + assert(!cast(cast(1.toDouble, TimestampType), DoubleType).resolved) + } } test("cast from string") { @@ -306,17 +321,34 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(cast(cast(cast( cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType), 5.toLong) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - 5.toShort) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - null) - checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), - ByteType), TimestampType), LongType), StringType), ShortType), - 5.toShort) + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + 5.toShort) + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + null) + checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), + ByteType), TimestampType), LongType), StringType), ShortType), + 5.toShort) + } + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { + assert(!cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType).resolved) + + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + null) + + assert(!cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), + ByteType), TimestampType), LongType), StringType), ShortType).resolved) + + } checkEvaluation(cast("23", DoubleType), 23d) checkEvaluation(cast("23", IntegerType), 23) @@ -377,29 +409,46 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(ts, LongType), 15.toLong) checkEvaluation(cast(ts, FloatType), 15.003f) checkEvaluation(cast(ts, DoubleType), 15.003) - checkEvaluation(cast(cast(tss, ShortType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, IntegerType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, LongType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), - millis.toFloat / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), - millis.toDouble / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), - Decimal(1)) - // A test for higher precision than millis - checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(tss, ShortType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, IntegerType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, LongType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), + millis.toFloat / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), + millis.toDouble / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), + Decimal(1)) + + // A test for higher precision than millis + checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + + checkEvaluation(cast(Double.NaN, TimestampType), null) + checkEvaluation(cast(1.0 / 0.0, TimestampType), null) + checkEvaluation(cast(Float.NaN, TimestampType), null) + checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + } - checkEvaluation(cast(Double.NaN, TimestampType), null) - checkEvaluation(cast(1.0 / 0.0, TimestampType), null) - checkEvaluation(cast(Float.NaN, TimestampType), null) - checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { + assert(!cast(cast(tss, ShortType), TimestampType).resolved) + assert(!cast(cast(tss, IntegerType), TimestampType).resolved) + assert(!cast(cast(tss, LongType), TimestampType).resolved) + assert(!cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType).resolved) + assert(!cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType).resolved) + assert(!cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT).resolved) + assert(!cast(cast(0.000001, TimestampType), DoubleType).resolved) + assert(!cast(Double.NaN, TimestampType).resolved) + assert(!cast(1.0 / 0.0, TimestampType).resolved) + assert(!cast(Float.NaN, TimestampType).resolved) + assert(!cast(1.0f / 0.0f, TimestampType).resolved) + } } test("cast from array") { @@ -1027,8 +1076,16 @@ class CastSuite extends CastSuiteBase { test("cast from int 2") { checkEvaluation(cast(1, LongType), 1.toLong) - checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) - checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) + checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + } + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { + assert(!cast(cast(1000, TimestampType), LongType).resolved) + assert(!cast(cast(-1200, TimestampType), LongType).resolved) + } checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) @@ -1314,7 +1371,7 @@ class CastSuite extends CastSuiteBase { test("SPARK-31710:fail casting from integral to timestamp by default") { Seq(true, false).foreach { enable => - withSQLConf(SQLConf.LEGACY_AllOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { assert(cast(2.toByte, TimestampType).resolved == enable) assert(cast(10.toShort, TimestampType).resolved == enable) assert(cast(3, TimestampType).resolved == enable) From 2c29c5b02c25258a5b75398d0b3d5e23af28e463 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 28 May 2020 10:24:55 +0800 Subject: [PATCH 09/75] fix test name --- docs/sql-migration-guide.md | 2 +- .../org/apache/spark/sql/catalyst/expressions/CastSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 8d11e6611e5f..b2cfd0124fd4 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -117,7 +117,7 @@ license: | - In Spark 3.0, when casting interval values to string type, there is no "interval" prefix, for example, `1 days 2 hours`. In Spark version 2.4 and below, the string contains the "interval" prefix like `interval 1 days 2 hours`. - In 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, for example, `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 below, when casting string to integrals and booleans, it does not trim the whitespaces from both ends; the foregoing results is `null`, while to datetimes, only the trailing spaces (= ASCII 32) are removed. - + ### Query Engine - In Spark version 2.4 and below, 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. These queries are treated as invalid in Spark 3.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 8a5697a50887..334d80d1f1ac 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 @@ -1369,7 +1369,7 @@ class CastSuite extends CastSuiteBase { } } - test("SPARK-31710:fail casting from integral to timestamp by default") { + test("SPARK-31710:fail casting from numeric to timestamp by default") { Seq(true, false).foreach { enable => withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { assert(cast(2.toByte, TimestampType).resolved == enable) From fd3efa2a4a229ca130d3232f6d36e7f178a006ca Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 28 May 2020 16:31:08 +0800 Subject: [PATCH 10/75] fix test case error --- .../src/test/resources/sql-tests/results/ansi/datetime.sql.out | 3 ++- sql/core/src/test/resources/sql-tests/results/datetime.sql.out | 3 ++- .../test/resources/sql-tests/results/postgreSQL/text.sql.out | 3 ++- .../test/resources/sql-tests/results/udf/udf-window.sql.out | 2 ++ sql/core/src/test/resources/sql-tests/results/window.sql.out | 3 ++- 5 files changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index aad1e5f34387..0ee7f996d5f8 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -21,7 +21,8 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema struct -- !query output -NULL 2016-12-31 2016-12-31 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query 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 a4f5b3772d2d..8227f11df8f6 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 @@ -15,7 +15,8 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema struct -- !query output -NULL 2016-12-31 2016-12-31 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query 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 811e7d6e4ca6..8e3f62ba1792 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 @@ -101,7 +101,8 @@ select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema struct -- !query output -123hellotruefalse2010-03-09 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 -- !query 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 a915c1bd6c71..fb2d5da57d82 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 @@ -17,6 +17,8 @@ AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot cast int to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 2 pos 37 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 625088f90ced..6514cead53cc 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 @@ -17,7 +17,8 @@ AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot cast int to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 2 pos 37 -- !query From 852d7306320f110a16b4d627c9e547a3183a4ef2 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 12:49:54 +0800 Subject: [PATCH 11/75] fix cmd to bash --- core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 39da57dd954a..b0f68880f1d8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -476,7 +476,7 @@ - + From 2e55be370685530ac98ee0aa9c9b4ab9c5b9ab96 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 14:54:04 +0800 Subject: [PATCH 12/75] fix scala style --- .../spark/sql/streaming/EventTimeWatermarkSuite.scala | 3 ++- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 6 ++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 8dd2f14b10e2..a5d7c5e3a866 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -185,7 +185,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // Trigger.Once to ensure that first and only trigger picks up the new data. testStream(aggWithWatermark)( - StartStream(Trigger.Once), // to make sure the query is not running when adding data 1st time + // to make sure the query is not running when adding data 1st time + StartStream(Trigger.Once), awaitTermination(), AddData(inputData, 15), 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 ba56d05a4360..fec5fdca0702 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 @@ -39,7 +39,8 @@ 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 originallegacyAllowCastNumericToTimestamp = TestHive.conf.legacyAllowCastNumericToTimestamp + private val originallegacyAllowCastNumericToTimestamp = + TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -70,7 +71,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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, originallegacyAllowCastNumericToTimestamp) + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + originallegacyAllowCastNumericToTimestamp) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) } finally { From 7637256038c483b35b615bfc824bcb3867d4cc9f Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 16:26:52 +0800 Subject: [PATCH 13/75] fix error test case --- .../resources/sql-tests/results/ansi/datetime.sql.out | 9 ++++----- .../resources/sql-tests/results/datetime-legacy.sql.out | 9 ++++----- .../test/resources/sql-tests/results/datetime.sql.out | 9 +++------ .../resources/sql-tests/results/postgreSQL/text.sql.out | 6 ++---- 4 files changed, 13 insertions(+), 20 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 5857a0ac90c7..2e6d4a2de832 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -79,9 +79,9 @@ select current_date = current_date(), current_timestamp = current_timestamp() -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct +struct<> -- !query output -NULL 2016-12-31 2016-12-31 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query @@ -811,10 +811,9 @@ NULL -- !query select to_date("16", "dd") -- !query schema -struct +struct<> -- !query output -1970-01-16 - +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query select to_date("02-29", "MM-dd") diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 8a726efafad8..73103be483df 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -73,9 +73,9 @@ true true -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct +struct<> -- !query output -NULL 2016-12-31 2016-12-31 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query @@ -783,10 +783,9 @@ NULL -- !query select to_date("16", "dd") -- !query schema -struct +struct<> -- !query output -1970-01-16 - +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query select to_date("02-29", "MM-dd") 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 0909f491a00d..c909c48f7818 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 @@ -75,9 +75,7 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema struct<[]> -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 - +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') @@ -784,10 +782,9 @@ NULL -- !query select to_date("16", "dd") -- !query schema -struct +struct<> -- !query output -1970-01-16 - +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query select to_date("02-29", "MM-dd") 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 bc6008513c1c..c2d1fd37ae40 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 @@ -101,8 +101,7 @@ select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema struct<[]> -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 -- !query @@ -118,8 +117,7 @@ select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema struct<[]> -- !query output -cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 48 - +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 48 -- !query select concat_ws(',',10,20,null,30) -- !query schema From 87cada78d0bc0cd5a6cb0c828b272bac6850cd1a Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 16:49:50 +0800 Subject: [PATCH 14/75] fix scala style --- .../test/resources/sql-tests/results/window.sql.out | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) 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 6514cead53cc..dbb033303cad 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 @@ -25,17 +25,9 @@ cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData ORDER BY cate, val -- !query schema -struct +struct<> -- !query output -NULL NULL 0 -3 NULL 1 -NULL a 0 -1 a 1 -1 a 1 -2 a 1 -1 b 1 -2 b 1 -3 b 1 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query From 13ae816b846744e72af4b5c1ff9b138207676419 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 19:34:45 +0800 Subject: [PATCH 15/75] fix fail test cases --- .../sql-tests/results/ansi/datetime.sql.out | 5 +- .../sql-tests/results/datetime-legacy.sql.out | 5 +- .../sql-tests/results/datetime.sql.out | 3 +- .../sql-tests/results/postgreSQL/text.sql.out | 8 +- .../sql-tests/results/udf/udf-window.sql.out | 13 +-- .../sql-tests/results/window.sql.out | 3 +- .../FlatMapGroupsWithStateSuite.scala | 82 ++++++++++--------- 7 files changed, 62 insertions(+), 57 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 2e6d4a2de832..b064c227fa1f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -79,9 +79,10 @@ select current_date = current_date(), current_timestamp = current_timestamp() -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<> +struct<[]> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 73103be483df..b2968e0e14c7 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -73,9 +73,10 @@ true true -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<> +struct<[]> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query 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 c909c48f7818..0be21ce11dff 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 @@ -75,7 +75,8 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema struct<[]> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') 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 c2d1fd37ae40..6f7dc01df290 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 @@ -101,7 +101,8 @@ select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema struct<[]> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 -- !query @@ -115,9 +116,10 @@ one -- !query select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema -struct<[]> +struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 48 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 48 -- !query select concat_ws(',',10,20,null,30) -- !query schema 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 fb2d5da57d82..b6a3b92b5c24 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 @@ -26,17 +26,10 @@ cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot 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 schema -struct +struct<> -- !query output -NULL NULL 0 -3 NULL 1 -NULL a 0 -1 a 1 -1 a 1 -2 a 1 -1 b 1 -2 b 1 -3 b 1 +org.apache.spark.sql.AnalysisException +cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 11 -- !query 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 dbb033303cad..b11241eb5f4d 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 @@ -27,7 +27,8 @@ ORDER BY cate, val -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query 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 ccd6e9f330b0..2b18a23f951d 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 @@ -806,51 +806,57 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { // Function to maintain the max event time as state and set the timeout timestamp based on the // current max event time seen. It returns the max event time in the state, or -1 if the state // was removed by timeout. - val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => { - assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 } - assertCanGetWatermark { state.getCurrentWatermarkMs() >= -1 } + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => { + assertCanGetProcessingTime { + state.getCurrentProcessingTimeMs() >= 0 + } + assertCanGetWatermark { + state.getCurrentWatermarkMs() >= -1 + } - val timeoutDelaySec = 5 - if (state.hasTimedOut) { - state.remove() - Iterator((key, -1)) - } else { - val valuesSeq = values.toSeq - val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) - val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec - state.update(maxEventTimeSec) - state.setTimeoutTimestamp(timeoutTimestampSec * 1000) - Iterator((key, maxEventTimeSec.toInt)) + val timeoutDelaySec = 5 + if (state.hasTimedOut) { + state.remove() + Iterator((key, -1)) + } else { + val valuesSeq = values.toSeq + val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) + val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec + state.update(maxEventTimeSec) + state.setTimeoutTimestamp(timeoutTimestampSec * 1000) + Iterator((key, maxEventTimeSec.toInt)) + } } - } - val inputData = MemoryStream[(String, Int)] - val result = - inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) - .withWatermark("eventTime", "10 seconds") - .as[(String, Long)] - .groupByKey(_._1) - .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) + val inputData = MemoryStream[(String, Int)] + val result = + inputData.toDS + .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .withWatermark("eventTime", "10 seconds") + .as[(String, Long)] + .groupByKey(_._1) + .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) - testStream(result, Update)( - StartStream(), + testStream(result, Update)( + StartStream(), - AddData(inputData, ("a", 11), ("a", 13), ("a", 15)), - // Max event time = 15. Timeout timestamp for "a" = 15 + 5 = 20. Watermark = 15 - 10 = 5. - CheckNewAnswer(("a", 15)), // Output = max event time of a + AddData(inputData, ("a", 11), ("a", 13), ("a", 15)), + // Max event time = 15. Timeout timestamp for "a" = 15 + 5 = 20. Watermark = 15 - 10 = 5. + CheckNewAnswer(("a", 15)), // Output = max event time of a - AddData(inputData, ("a", 4)), // Add data older than watermark for "a" - CheckNewAnswer(), // No output as data should get filtered by watermark + AddData(inputData, ("a", 4)), // Add data older than watermark for "a" + CheckNewAnswer(), // No output as data should get filtered by watermark - AddData(inputData, ("a", 10)), // Add data newer than watermark for "a" - CheckNewAnswer(("a", 15)), // Max event time is still the same - // Timeout timestamp for "a" is still 20 as max event time for "a" is still 15. - // Watermark is still 5 as max event time for all data is still 15. + AddData(inputData, ("a", 10)), // Add data newer than watermark for "a" + CheckNewAnswer(("a", 15)), // Max event time is still the same + // Timeout timestamp for "a" is still 20 as max event time for "a" is still 15. + // Watermark is still 5 as max event time for all data is still 15. - AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a" - // Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20. - CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1 - ) + AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a" + // Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20. + CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1 + ) + } } test("flatMapGroupsWithState - uses state format version 2 by default") { From 4abd8d3f6ce501a4454009c149ad9c13a879d8c4 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 20:32:27 +0800 Subject: [PATCH 16/75] fix scala style --- .../sql-tests/results/ansi/datetime.sql.out | 4 ++-- .../sql-tests/results/datetime-legacy.sql.out | 4 ++-- .../resources/sql-tests/results/datetime.sql.out | 4 ++-- .../test/resources/sql-tests/results/window.sql.out | 12 ++---------- 4 files changed, 8 insertions(+), 16 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index b064c227fa1f..446608754cda 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -819,9 +819,9 @@ java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisExceptio -- !query select to_date("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index b2968e0e14c7..bc14e6253b6d 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -791,9 +791,9 @@ java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisExceptio -- !query select to_date("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query 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 0be21ce11dff..e4265144c2e2 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 @@ -790,9 +790,9 @@ java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisExceptio -- !query select to_date("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query 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 b11241eb5f4d..c10829c64342 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 @@ -35,17 +35,9 @@ cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, 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 schema -struct +struct<> -- !query output -NULL NULL 3 -3 NULL 3 -NULL a 1 -1 a 2 -1 a 4 -2 a 4 -1 b 3 -2 b 6 -3 b 6 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query From b13316fbd922b7239728951cf8d13cfed2e552bf Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 29 May 2020 22:52:51 +0800 Subject: [PATCH 17/75] fix error test case --- docs/sql-migration-guide.md | 4 +++- .../catalyst/expressions/datetimeExpressions.scala | 2 -- .../test/resources/sql-tests/results/window.sql.out | 12 ++---------- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 7fcdd1c363c0..a9ae3c877d11 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -28,7 +28,9 @@ license: | - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. - - In Spark 3.1, casting numeric to timestamp will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. + - In Spark 3.1, casting numeric to timestamp and will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. + + - In Spark 3.1, to_date function with date format as the second parameter will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. ## Upgrading from Spark SQL 2.4 to 3.0 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 7dc008a2e5df..c54abf834c42 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 @@ -1617,8 +1617,6 @@ case class ToUTCTimestamp(left: Expression, right: Expression) Examples: > SELECT _FUNC_('2009-07-30 04:17:52'); 2009-07-30 - > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd'); - 2016-12-31 """, group = "datetime_funcs", since = "1.5.0") 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 c10829c64342..7da5b3db2166 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 @@ -54,17 +54,9 @@ cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData ORDER BY cate, val -- !query schema -struct +struct<> -- !query output -NULL NULL 0 -3 NULL 1 -NULL a 0 -1 a 2 -1 a 2 -2 a 3 -1 b 1 -2 b 2 -3 b 2 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query From 50cc0cfa5085faaecd9a70ad9f10a5cecb035052 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 30 May 2020 07:55:10 +0800 Subject: [PATCH 18/75] fix scala style --- .../sql-tests/results/ansi/datetime.sql.out | 2 +- .../sql-tests/results/datetime-legacy.sql.out | 2 +- .../resources/sql-tests/results/datetime.sql.out | 2 +- .../sql-tests/results/postgreSQL/text.sql.out | 2 +- .../sql-tests/results/udf/udf-window.sql.out | 13 +++---------- .../test/resources/sql-tests/results/window.sql.out | 3 ++- 6 files changed, 9 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 446608754cda..bed53341cc62 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -79,7 +79,7 @@ select current_date = current_date(), current_timestamp = current_timestamp() -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<[]> +struct<> -- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index bc14e6253b6d..9319c1bef2b1 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -73,7 +73,7 @@ true true -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<[]> +struct<> -- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 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 e4265144c2e2..e9511c259451 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 @@ -73,7 +73,7 @@ true true -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<[]> +struct<> -- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 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 6f7dc01df290..16367a37a697 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 @@ -99,7 +99,7 @@ one -- !query select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema -struct<[]> +struct<> -- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 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 b6a3b92b5c24..136f489ffb29 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 @@ -36,17 +36,10 @@ cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, 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 schema -struct +struct<> -- !query output -NULL NULL 3 -3 NULL 3 -NULL a 1 -1 a 2 -1 a 4 -2 a 4 -1 b 3 -2 b 6 -3 b 6 +org.apache.spark.sql.AnalysisException +cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 11 -- !query 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 7da5b3db2166..f7b8861917cf 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 @@ -37,7 +37,8 @@ ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, v -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query From 22638e4fc2349b94dd4f286edf8d0e9fce076f78 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 30 May 2020 08:41:22 +0800 Subject: [PATCH 19/75] fix test case error --- .../test/resources/sql-tests/results/window.sql.out | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) 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 f7b8861917cf..48cee0aed25b 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 @@ -64,17 +64,9 @@ java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisExceptio 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 schema -struct +struct<> -- !query output -NULL NULL NULL -3 NULL 3 -NULL a NULL -1 a 4 -1 a 4 -2 a 2 -1 b 3 -2 b 5 -3 b 3 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query From b1dda01e65ac16c0d656672b3756c0a132a1e85c Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 30 May 2020 17:48:04 +0800 Subject: [PATCH 20/75] fix test case error --- .../sql-tests/results/ansi/datetime.sql.out | 6 +- .../sql-tests/results/datetime-legacy.sql.out | 6 +- .../sql-tests/results/datetime.sql.out | 6 +- .../sql-tests/results/window.sql.out | 17 +- .../apache/spark/sql/DateFunctionsSuite.scala | 77 +- .../spark/sql/StatisticsCollectionSuite.scala | 81 +- .../streaming/MicroBatchExecutionSuite.scala | 78 +- .../sources/ForeachWriterSuite.scala | 159 ++-- .../sql/expressions/ExpressionInfoSuite.scala | 109 +-- .../streaming/EventTimeWatermarkSuite.scala | 804 +++++++++--------- .../sql/streaming/FileStreamSinkSuite.scala | 89 +- .../FlatMapGroupsWithStateSuite.scala | 300 ++++--- .../streaming/StreamingAggregationSuite.scala | 205 ++--- .../StreamingDeduplicationSuite.scala | 230 +++-- .../sql/hive/execution/HiveQuerySuite.scala | 1 + .../sql/hive/execution/HiveUDFSuite.scala | 16 +- 16 files changed, 1070 insertions(+), 1114 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index bed53341cc62..e254a3a6471d 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -814,14 +814,16 @@ select to_date("16", "dd") -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query select to_date("02-29", "MM-dd") -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 9319c1bef2b1..85a78e3307a1 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -786,14 +786,16 @@ select to_date("16", "dd") -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query select to_date("02-29", "MM-dd") -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query 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 e9511c259451..dc9c33cf3159 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 @@ -785,14 +785,16 @@ select to_date("16", "dd") -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query select to_date("02-29", "MM-dd") -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 -- !query 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 48cee0aed25b..f6495f1dc8df 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 @@ -48,7 +48,7 @@ ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, struct<> -- !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 +cannot resolve '`val_long`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query @@ -57,7 +57,8 @@ ORDER BY cate, val -- !query schema struct<> -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +org.apache.spark.sql.AnalysisException +cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query @@ -73,17 +74,9 @@ java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisExceptio 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 schema -struct +struct<> -- !query output -NULL NULL NULL -1 NULL 1 -1 a 4 -1 a 4 -2 a 2147483652 -2147483650 a 2147483650 -NULL b NULL -3 b 2147483653 -2147483650 b 2147483650 +java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val_long`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 -- !query 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 0a01514acb7e..a99377f83eda 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,6 +23,8 @@ import java.time.{Instant, LocalDateTime, ZoneId} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit +import org.scalatest.BeforeAndAfter + import org.apache.spark.{SparkException, SparkUpgradeException} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CEST, LA} import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -32,9 +34,17 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.DoubleType import org.apache.spark.unsafe.types.CalendarInterval -class DateFunctionsSuite extends QueryTest with SharedSparkSession { +class DateFunctionsSuite extends QueryTest with SharedSparkSession with BeforeAndAfter{ import testImplicits._ + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } test("function current_date") { val df1 = Seq((1, 2), (3, 1)).toDF("a", "b") val d0 = DateTimeUtils.currentDate(ZoneId.systemDefault()) @@ -422,39 +432,38 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { df.selectExpr("to_date(s)"), Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - // now with format - checkAnswer( - df.select(to_date(col("t"), "yyyy-MM-dd")), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), - Row(Date.valueOf("2014-12-31")))) - checkAnswer( - df.select(to_date(col("d"), "yyyy-MM-dd")), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")), - Row(Date.valueOf("2014-12-31")))) - val confKey = SQLConf.LEGACY_TIME_PARSER_POLICY.key - withSQLConf(confKey -> "corrected") { - checkAnswer( - df.select(to_date(col("s"), "yyyy-MM-dd")), - Seq(Row(null), Row(Date.valueOf("2014-12-31")), Row(null))) - } - withSQLConf(confKey -> "exception") { - checkExceptionMessage(df.select(to_date(col("s"), "yyyy-MM-dd"))) - } - // now switch format - checkAnswer( - df.select(to_date(col("s"), "yyyy-dd-MM")), - Seq(Row(null), Row(null), Row(Date.valueOf("2014-12-31")))) - - // invalid format + // now with format + checkAnswer( + df.select(to_date(col("t"), "yyyy-MM-dd")), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("d"), "yyyy-MM-dd")), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + val confKey = SQLConf.LEGACY_TIME_PARSER_POLICY.key + withSQLConf(confKey -> "corrected") { checkAnswer( - df.select(to_date(col("s"), "yyyy-hh-MM")), - Seq(Row(null), Row(null), Row(null))) - val e = intercept[SparkUpgradeException](df.select(to_date(col("s"), "yyyy-dd-aa")).collect()) - assert(e.getCause.isInstanceOf[IllegalArgumentException]) - assert(e.getMessage.contains("You may get a different result due to the upgrading of Spark")) + df.select(to_date(col("s"), "yyyy-MM-dd")), + Seq(Row(null), Row(Date.valueOf("2014-12-31")), Row(null))) + } + withSQLConf(confKey -> "exception") { + checkExceptionMessage(df.select(to_date(col("s"), "yyyy-MM-dd"))) } + // now switch format + checkAnswer( + df.select(to_date(col("s"), "yyyy-dd-MM")), + Seq(Row(null), Row(null), Row(Date.valueOf("2014-12-31")))) + + // invalid format + checkAnswer( + df.select(to_date(col("s"), "yyyy-hh-MM")), + Seq(Row(null), Row(null), Row(null))) + val e = intercept[SparkUpgradeException](df.select(to_date(col("s"), "yyyy-dd-aa")).collect()) + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + assert(e.getMessage.contains("You may get a different result due to the upgrading of Spark")) + // february val x1 = "2016-02-29" val x2 = "2017-02-29" @@ -573,8 +582,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { test("unix_timestamp") { Seq("corrected", "legacy").foreach { legacyParserPolicy => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, - SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { 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") @@ -700,8 +708,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { test("to_timestamp") { Seq("legacy", "corrected").foreach { legacyParserPolicy => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, - SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { 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") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 03c99f52284b..1f5fc17efdf8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -22,6 +22,7 @@ import java.net.URI import java.util.TimeZone import java.util.concurrent.TimeUnit +import org.scalatest.BeforeAndAfter import scala.collection.mutable import org.apache.spark.sql.catalyst.TableIdentifier @@ -39,9 +40,19 @@ import org.apache.spark.util.Utils /** * End-to-end suite testing statistics collection and use on both entire table and columns. */ -class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession { +class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession + with BeforeAndAfter{ import testImplicits._ + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } + test("estimates the size of a limit 0 on outer join") { withTempView("test") { Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") @@ -454,44 +465,42 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } test("store and retrieve column stats in different time zones") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val (start, end) = (0, TimeUnit.DAYS.toSeconds(2)) - - def checkTimestampStats( - t: DataType, - srcTimeZone: TimeZone, - dstTimeZone: TimeZone)(checker: ColumnStat => Unit): Unit = { - val table = "time_table" - val column = "T" - val original = TimeZone.getDefault - try { - withTable(table) { - TimeZone.setDefault(srcTimeZone) - spark.range(start, end) - .select('id.cast(TimestampType).cast(t).as(column)) - .write.saveAsTable(table) - sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") - - TimeZone.setDefault(dstTimeZone) - val stats = getCatalogTable(table) - .stats.get.colStats(column).toPlanStat(column, t) - checker(stats) - } - } finally { - TimeZone.setDefault(original) + val (start, end) = (0, TimeUnit.DAYS.toSeconds(2)) + + def checkTimestampStats( + t: DataType, + srcTimeZone: TimeZone, + dstTimeZone: TimeZone)(checker: ColumnStat => Unit): Unit = { + val table = "time_table" + val column = "T" + val original = TimeZone.getDefault + try { + withTable(table) { + TimeZone.setDefault(srcTimeZone) + spark.range(start, end) + .select('id.cast(TimestampType).cast(t).as(column)) + .write.saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") + + TimeZone.setDefault(dstTimeZone) + val stats = getCatalogTable(table) + .stats.get.colStats(column).toPlanStat(column, t) + checker(stats) } + } finally { + TimeZone.setDefault(original) } + } - DateTimeTestUtils.outstandingZoneIds.foreach { zid => - val timeZone = TimeZone.getTimeZone(zid) - checkTimestampStats(DateType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => - assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) - assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) - } - checkTimestampStats(TimestampType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => - assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) - assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) - } + DateTimeTestUtils.outstandingZoneIds.foreach { zid => + val timeZone = TimeZone.getTimeZone(zid) + checkTimestampStats(DateType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => + assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) + assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) + } + checkTimestampStats(TimestampType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => + assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) + assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala index bffc764f227b..354592b7fdf2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala @@ -27,48 +27,52 @@ class MicroBatchExecutionSuite extends StreamTest with BeforeAndAfter { import testImplicits._ import org.apache.spark.sql.internal.SQLConf + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) sqlContext.streams.active.foreach(_.stop()) } test("SPARK-24156: do not plan a no-data batch again after it has already been planned") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val df = 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]) + val inputData = MemoryStream[Int] + val df = 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]) - testStream(df)( - AddData(inputData, 10, 11, 12, 13, 14, 15), // Set watermark to 5 - CheckAnswer(), - AddData(inputData, 25), // Set watermark to 15 to make MicroBatchExecution run no-data batch - CheckAnswer((10, 5)), // Last batch should be a no-data batch - StopStream, - Execute { q => - // Delete the last committed batch from the commit log to signify that the last batch - // (a no-data batch) never completed - val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) - q.commitLog.purgeAfter(commit - 1) - }, - // Add data before start so that MicroBatchExecution can plan a batch. It should not, - // it should first re-run the incomplete no-data batch and then run a new batch to process - // new data. - AddData(inputData, 30), - StartStream(), - CheckNewAnswer((15, 1)), // This should not throw the error reported in SPARK-24156 - StopStream, - Execute { q => - // Delete the entire commit log - val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) - q.commitLog.purge(commit + 1) - }, - AddData(inputData, 50), - StartStream(), - CheckNewAnswer((25, 1), (30, 1)) // This should not throw the error reported in SPARK-24156 - ) - } + testStream(df)( + AddData(inputData, 10, 11, 12, 13, 14, 15), // Set watermark to 5 + CheckAnswer(), + AddData(inputData, 25), // Set watermark to 15 to make MicroBatchExecution run no-data batch + CheckAnswer((10, 5)), // Last batch should be a no-data batch + StopStream, + Execute { q => + // Delete the last committed batch from the commit log to signify that the last batch + // (a no-data batch) never completed + val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) + q.commitLog.purgeAfter(commit - 1) + }, + // Add data before start so that MicroBatchExecution can plan a batch. It should not, + // it should first re-run the incomplete no-data batch and then run a new batch to process + // new data. + AddData(inputData, 30), + StartStream(), + CheckNewAnswer((15, 1)), // This should not throw the error reported in SPARK-24156 + StopStream, + Execute { q => + // Delete the entire commit log + val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) + q.commitLog.purge(commit + 1) + }, + AddData(inputData, 50), + StartStream(), + CheckNewAnswer((25, 1), (30, 1)) // This should not throw the error reported in SPARK-24156 + ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index e108f14df18f..34f00750dd3c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -34,7 +34,14 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA import testImplicits._ import org.apache.spark.sql.internal.SQLConf + + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) sqlContext.streams.active.foreach(_.stop()) } @@ -160,91 +167,87 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA } test("foreach with watermark: complete") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - - val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "10 seconds") - .groupBy(window($"eventTime", "5 seconds") as 'window) - .agg(count("*") as 'count) - .select($"count".as[Long]) - .map(_.toInt) - .repartition(1) - - val query = windowedAggregation - .writeStream - .outputMode(OutputMode.Complete) - .foreach(new TestForeachWriter()) - .start() - try { - inputData.addData(10, 11, 12) - query.processAllAvailable() + val inputData = MemoryStream[Int] - val allEvents = ForeachWriterSuite.allEvents() - assert(allEvents.size === 1) - val expectedEvents = Seq( - ForeachWriterSuite.Open(partition = 0, version = 0), - ForeachWriterSuite.Process(value = 3), - ForeachWriterSuite.Close(None) - ) - assert(allEvents === Seq(expectedEvents)) - } finally { - query.stop() - } + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"count".as[Long]) + .map(_.toInt) + .repartition(1) + + val query = windowedAggregation + .writeStream + .outputMode(OutputMode.Complete) + .foreach(new TestForeachWriter()) + .start() + try { + inputData.addData(10, 11, 12) + query.processAllAvailable() + + val allEvents = ForeachWriterSuite.allEvents() + assert(allEvents.size === 1) + val expectedEvents = Seq( + ForeachWriterSuite.Open(partition = 0, version = 0), + ForeachWriterSuite.Process(value = 3), + ForeachWriterSuite.Close(None) + ) + assert(allEvents === Seq(expectedEvents)) + } finally { + query.stop() } } test("foreach with watermark: append") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - - val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "10 seconds") - .groupBy(window($"eventTime", "5 seconds") as 'window) - .agg(count("*") as 'count) - .select($"count".as[Long]) - .map(_.toInt) - .repartition(1) - - val query = windowedAggregation - .writeStream - .outputMode(OutputMode.Append) - .foreach(new TestForeachWriter()) - .start() - try { - inputData.addData(10, 11, 12) - query.processAllAvailable() - inputData.addData(25) // Evict items less than previous watermark - query.processAllAvailable() + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"count".as[Long]) + .map(_.toInt) + .repartition(1) + + val query = windowedAggregation + .writeStream + .outputMode(OutputMode.Append) + .foreach(new TestForeachWriter()) + .start() + try { + inputData.addData(10, 11, 12) + query.processAllAvailable() + inputData.addData(25) // Evict items less than previous watermark + query.processAllAvailable() - // There should be 3 batches and only does the last batch contain a value. - val allEvents = ForeachWriterSuite.allEvents() - assert(allEvents.size === 4) - val expectedEvents = Seq( - Seq( - ForeachWriterSuite.Open(partition = 0, version = 0), - ForeachWriterSuite.Close(None) - ), - Seq( - ForeachWriterSuite.Open(partition = 0, version = 1), - ForeachWriterSuite.Close(None) - ), - Seq( - ForeachWriterSuite.Open(partition = 0, version = 2), - ForeachWriterSuite.Close(None) - ), - Seq( - ForeachWriterSuite.Open(partition = 0, version = 3), - ForeachWriterSuite.Process(value = 3), - ForeachWriterSuite.Close(None) - ) + // There should be 3 batches and only does the last batch contain a value. + val allEvents = ForeachWriterSuite.allEvents() + assert(allEvents.size === 4) + val expectedEvents = Seq( + Seq( + ForeachWriterSuite.Open(partition = 0, version = 0), + ForeachWriterSuite.Close(None) + ), + Seq( + ForeachWriterSuite.Open(partition = 0, version = 1), + ForeachWriterSuite.Close(None) + ), + Seq( + ForeachWriterSuite.Open(partition = 0, version = 2), + ForeachWriterSuite.Close(None) + ), + Seq( + ForeachWriterSuite.Open(partition = 0, version = 3), + ForeachWriterSuite.Process(value = 3), + ForeachWriterSuite.Close(None) ) - assert(allEvents === expectedEvents) - } finally { - query.stop() - } + ) + assert(allEvents === expectedEvents) + } finally { + query.stop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index 4c9ff31d6463..140d54450dbc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.expressions +import org.scalatest.BeforeAndAfter import scala.collection.parallel.immutable.ParVector import org.apache.spark.SparkFunSuite @@ -26,7 +27,16 @@ import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { +class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession with BeforeAndAfter { + + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } test("Replace _FUNC_ in ExpressionInfo") { val info = spark.sessionState.catalog.lookupFunctionInfo(FunctionIdentifier("upper")) @@ -105,57 +115,52 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { } test("check outputs of expression examples") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - def unindentAndTrim(s: String): String = { - s.replaceAll("\n\\s+", "\n").trim - } - - val beginSqlStmtRe = " > ".r - val endSqlStmtRe = ";\n".r - - def checkExampleSyntax(example: String): Unit = { - val beginStmtNum = beginSqlStmtRe.findAllIn(example).length - val endStmtNum = endSqlStmtRe.findAllIn(example).length - assert(beginStmtNum === endStmtNum, - "The number of ` > ` does not match to the number of `;`") - } - - val exampleRe = """^(.+);\n(?s)(.+)$""".r - val ignoreSet = Set( - // One of examples shows getting the current timestamp - "org.apache.spark.sql.catalyst.expressions.UnixTimestamp", - "org.apache.spark.sql.catalyst.expressions.CurrentDate", - "org.apache.spark.sql.catalyst.expressions.CurrentTimestamp", - "org.apache.spark.sql.catalyst.expressions.Now", - // Random output without a seed - "org.apache.spark.sql.catalyst.expressions.Rand", - "org.apache.spark.sql.catalyst.expressions.Randn", - "org.apache.spark.sql.catalyst.expressions.Shuffle", - "org.apache.spark.sql.catalyst.expressions.Uuid", - // The example calls methods that return unstable results. - "org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection") - - 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() - // Coalescing partitions can change result order, so disable it. - clonedSpark.sessionState.conf.setConf(SQLConf.COALESCE_PARTITIONS_ENABLED, false) - 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 { - 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 _ => - } + def unindentAndTrim(s: String): String = { + s.replaceAll("\n\\s+", "\n").trim + } + val beginSqlStmtRe = " > ".r + val endSqlStmtRe = ";\n".r + def checkExampleSyntax(example: String): Unit = { + val beginStmtNum = beginSqlStmtRe.findAllIn(example).length + val endStmtNum = endSqlStmtRe.findAllIn(example).length + assert(beginStmtNum === endStmtNum, + "The number of ` > ` does not match to the number of `;`") + } + val exampleRe = """^(.+);\n(?s)(.+)$""".r + val ignoreSet = Set( + // One of examples shows getting the current timestamp + "org.apache.spark.sql.catalyst.expressions.UnixTimestamp", + "org.apache.spark.sql.catalyst.expressions.CurrentDate", + "org.apache.spark.sql.catalyst.expressions.CurrentTimestamp", + "org.apache.spark.sql.catalyst.expressions.Now", + // Random output without a seed + "org.apache.spark.sql.catalyst.expressions.Rand", + "org.apache.spark.sql.catalyst.expressions.Randn", + "org.apache.spark.sql.catalyst.expressions.Shuffle", + "org.apache.spark.sql.catalyst.expressions.Uuid", + // The example calls methods that return unstable results. + "org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection") + + 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() + // Coalescing partitions can change result order, so disable it. + clonedSpark.sessionState.conf.setConf(SQLConf.COALESCE_PARTITIONS_ENABLED, false) + 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 { + 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/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index a5d7c5e3a866..eaae20397822 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -41,7 +41,13 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche import testImplicits._ + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) sqlContext.streams.active.foreach(_.stop()) } @@ -127,109 +133,103 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("event time and watermark metrics") { // No event time metrics when there is no watermarking - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData1 = MemoryStream[Int] - val aggWithoutWatermark = inputData1.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) - .groupBy(window($"eventTime", "5 seconds") as 'window) - .agg(count("*") as 'count) - .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) - - testStream(aggWithoutWatermark, outputMode = Complete)( - AddData(inputData1, 15), - CheckAnswer((15, 1)), - assertEventStats { e => assert(e.isEmpty) }, - AddData(inputData1, 10, 12, 14), - CheckAnswer((10, 3), (15, 1)), - assertEventStats { e => assert(e.isEmpty) } - ) + val inputData1 = MemoryStream[Int] + val aggWithoutWatermark = inputData1.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(aggWithoutWatermark, outputMode = Complete)( + AddData(inputData1, 15), + CheckAnswer((15, 1)), + assertEventStats { e => assert(e.isEmpty) }, + AddData(inputData1, 10, 12, 14), + CheckAnswer((10, 3), (15, 1)), + assertEventStats { e => assert(e.isEmpty) } + ) - // All event time metrics where watermarking is set - val inputData2 = MemoryStream[Int] - val aggWithWatermark = inputData2.toDF() + // All event time metrics where watermarking is set + val inputData2 = MemoryStream[Int] + val aggWithWatermark = inputData2.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]) - testStream(aggWithWatermark)( - AddData(inputData2, 15), - CheckAnswer(), - assertEventStats(min = 15, max = 15, avg = 15, wtrmark = 0), - AddData(inputData2, 10, 12, 14), - CheckAnswer(), - assertEventStats(min = 10, max = 14, avg = 12, wtrmark = 5), - AddData(inputData2, 25), - CheckAnswer((10, 3)), - assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5) - ) - } + testStream(aggWithWatermark)( + AddData(inputData2, 15), + CheckAnswer(), + assertEventStats(min = 15, max = 15, avg = 15, wtrmark = 0), + AddData(inputData2, 10, 12, 14), + CheckAnswer(), + assertEventStats(min = 10, max = 14, avg = 12, wtrmark = 5), + AddData(inputData2, 25), + CheckAnswer((10, 3)), + assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5) + ) } test("event time and watermark metrics with Trigger.Once (SPARK-24699)") { // All event time metrics where watermarking is set - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val aggWithWatermark = inputData.toDF() + 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]) - // Unlike the ProcessingTime trigger, Trigger.Once only runs one trigger every time - // the query is started and it does not run no-data batches. Hence the answer generated - // by the updated watermark is only generated the next time the query is started. - // Also, the data to process in the next trigger is added *before* starting the stream in - // Trigger.Once to ensure that first and only trigger picks up the new data. - - testStream(aggWithWatermark)( - // to make sure the query is not running when adding data 1st time - StartStream(Trigger.Once), - awaitTermination(), - - AddData(inputData, 15), - StartStream(Trigger.Once), - awaitTermination(), - CheckNewAnswer(), - assertEventStats(min = 15, max = 15, avg = 15, wtrmark = 0), - // watermark should be updated to 15 - 10 = 5 - - AddData(inputData, 10, 12, 14), - StartStream(Trigger.Once), - awaitTermination(), - CheckNewAnswer(), - assertEventStats(min = 10, max = 14, avg = 12, wtrmark = 5), - // watermark should stay at 5 - - AddData(inputData, 25), - StartStream(Trigger.Once), - awaitTermination(), - CheckNewAnswer(), - assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5), - // watermark should be updated to 25 - 10 = 15 - - AddData(inputData, 50), - StartStream(Trigger.Once), - awaitTermination(), - CheckNewAnswer((10, 3)), // watermark = 15 is used to generate this - assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 15), - // watermark should be updated to 50 - 10 = 40 - - AddData(inputData, 50), - StartStream(Trigger.Once), - awaitTermination(), - CheckNewAnswer((15, 1), (25, 1)), // watermark = 40 is used to generate this - assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 40)) - } + // Unlike the ProcessingTime trigger, Trigger.Once only runs one trigger every time + // the query is started and it does not run no-data batches. Hence the answer generated + // by the updated watermark is only generated the next time the query is started. + // Also, the data to process in the next trigger is added *before* starting the stream in + // Trigger.Once to ensure that first and only trigger picks up the new data. + + testStream(aggWithWatermark)( + StartStream(Trigger.Once), // to make sure the query is not running when adding data 1st time + awaitTermination(), + + AddData(inputData, 15), + StartStream(Trigger.Once), + awaitTermination(), + CheckNewAnswer(), + assertEventStats(min = 15, max = 15, avg = 15, wtrmark = 0), + // watermark should be updated to 15 - 10 = 5 + + AddData(inputData, 10, 12, 14), + StartStream(Trigger.Once), + awaitTermination(), + CheckNewAnswer(), + assertEventStats(min = 10, max = 14, avg = 12, wtrmark = 5), + // watermark should stay at 5 + + AddData(inputData, 25), + StartStream(Trigger.Once), + awaitTermination(), + CheckNewAnswer(), + assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5), + // watermark should be updated to 25 - 10 = 15 + + AddData(inputData, 50), + StartStream(Trigger.Once), + awaitTermination(), + CheckNewAnswer((10, 3)), // watermark = 15 is used to generate this + assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 15), + // watermark should be updated to 50 - 10 = 40 + + AddData(inputData, 50), + StartStream(Trigger.Once), + awaitTermination(), + CheckNewAnswer((15, 1), (25, 1)), // watermark = 40 is used to generate this + assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 40)) } test("recovery from Spark ver 2.3.1 commit log without commit metadata (SPARK-24699)") { // All event time metrics where watermarking is set - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val aggWithWatermark = inputData.toDF() + val inputData = MemoryStream[Int] + val aggWithWatermark = inputData.toDF() .withColumn("eventTime", $"value".cast("timestamp")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) @@ -237,19 +237,19 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) - val resourceUri = this.getClass.getResource( - "/structured-streaming/checkpoint-version-2.3.1-without-commit-log-metadata/").toURI + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-2.3.1-without-commit-log-metadata/").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) + 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) - inputData.addData(15) - inputData.addData(10, 12, 14) + inputData.addData(15) + inputData.addData(10, 12, 14) - testStream(aggWithWatermark)( - /* + testStream(aggWithWatermark)( + /* Note: The checkpoint was generated using the following input in Spark version 2.3.1 @@ -264,280 +264,265 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // Offset log should have watermark recorded as 5. */ - StartStream(Trigger.Once), - awaitTermination(), + StartStream(Trigger.Once), + awaitTermination(), - AddData(inputData, 25), - StartStream(Trigger.Once, checkpointLocation = checkpointDir.getAbsolutePath), - awaitTermination(), - CheckNewAnswer(), - assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5), - // watermark should be updated to 25 - 10 = 15 - - AddData(inputData, 50), - StartStream(Trigger.Once, checkpointLocation = checkpointDir.getAbsolutePath), - awaitTermination(), - CheckNewAnswer((10, 3)), // watermark = 15 is used to generate this - assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 15), - // watermark should be updated to 50 - 10 = 40 - - AddData(inputData, 50), - StartStream(Trigger.Once, checkpointLocation = checkpointDir.getAbsolutePath), - awaitTermination(), - CheckNewAnswer((15, 1), (25, 1)), // watermark = 40 is used to generate this - assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 40)) - } + AddData(inputData, 25), + StartStream(Trigger.Once, checkpointLocation = checkpointDir.getAbsolutePath), + awaitTermination(), + CheckNewAnswer(), + assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5), + // watermark should be updated to 25 - 10 = 15 + + AddData(inputData, 50), + StartStream(Trigger.Once, checkpointLocation = checkpointDir.getAbsolutePath), + awaitTermination(), + CheckNewAnswer((10, 3)), // watermark = 15 is used to generate this + assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 15), + // watermark should be updated to 50 - 10 = 40 + + AddData(inputData, 50), + StartStream(Trigger.Once, checkpointLocation = checkpointDir.getAbsolutePath), + awaitTermination(), + CheckNewAnswer((15, 1), (25, 1)), // watermark = 40 is used to generate this + assertEventStats(min = 50, max = 50, avg = 50, wtrmark = 40)) } test("append mode") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int] - val windowedAggregation = 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]) - - testStream(windowedAggregation)( - AddData(inputData, 10, 11, 12, 13, 14, 15), - CheckNewAnswer(), - AddData(inputData, 25), // Advance watermark to 15 seconds - CheckNewAnswer((10, 5)), - assertNumStateRows(2), - AddData(inputData, 10), // Should not emit anything as data less than watermark - CheckNewAnswer(), - assertNumStateRows(2) - ) - } + val windowedAggregation = 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]) + + testStream(windowedAggregation)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckNewAnswer(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckNewAnswer((10, 5)), + assertNumStateRows(2), + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckNewAnswer(), + assertNumStateRows(2) + ) } test("update mode") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") + val inputData = MemoryStream[Int] + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") - val windowedAggregation = 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]) - - testStream(windowedAggregation, OutputMode.Update)( - AddData(inputData, 10, 11, 12, 13, 14, 15), - CheckNewAnswer((10, 5), (15, 1)), - AddData(inputData, 25), // Advance watermark to 15 seconds - CheckNewAnswer((25, 1)), - assertNumStateRows(2), - AddData(inputData, 10, 25), // Ignore 10 as its less than watermark - CheckNewAnswer((25, 2)), - assertNumStateRows(2), - AddData(inputData, 10), // Should not emit anything as data less than watermark - CheckNewAnswer(), - assertNumStateRows(2) - ) - } + val windowedAggregation = 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]) + + testStream(windowedAggregation, OutputMode.Update)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckNewAnswer((10, 5), (15, 1)), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckNewAnswer((25, 1)), + assertNumStateRows(2), + AddData(inputData, 10, 25), // Ignore 10 as its less than watermark + CheckNewAnswer((25, 2)), + assertNumStateRows(2), + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckNewAnswer(), + assertNumStateRows(2) + ) } test("delay in months and years handled correctly") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val currentTimeMs = System.currentTimeMillis - val currentTime = new Date(currentTimeMs) + val currentTimeMs = System.currentTimeMillis + val currentTime = new Date(currentTimeMs) - val input = MemoryStream[Long] - val aggWithWatermark = input.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "2 years 5 months") - .groupBy(window($"eventTime", "5 seconds") as 'window) - .agg(count("*") as 'count) - .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + val input = MemoryStream[Long] + val aggWithWatermark = input.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "2 years 5 months") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + def monthsSinceEpoch(date: Date): Int = { + val cal = Calendar.getInstance() + cal.setTime(date) + cal.get(Calendar.YEAR) * 12 + cal.get(Calendar.MONTH) + } - def monthsSinceEpoch(date: Date): Int = { - val cal = Calendar.getInstance() - cal.setTime(date) - cal.get(Calendar.YEAR) * 12 + cal.get(Calendar.MONTH) + testStream(aggWithWatermark)( + AddData(input, MILLISECONDS.toSeconds(currentTimeMs)), + CheckAnswer(), + AddData(input, MILLISECONDS.toSeconds(currentTimeMs)), + CheckAnswer(), + assertEventStats { e => + assert(timestampFormat.parse(e.get("max")).getTime === + SECONDS.toMillis(MILLISECONDS.toSeconds((currentTimeMs)))) + val watermarkTime = timestampFormat.parse(e.get("watermark")) + val monthDiff = monthsSinceEpoch(currentTime) - monthsSinceEpoch(watermarkTime) + // monthsSinceEpoch is like `math.floor(num)`, so monthDiff has two possible values. + assert(monthDiff === 29 || monthDiff === 30, + s"currentTime: $currentTime, watermarkTime: $watermarkTime") } - - testStream(aggWithWatermark)( - AddData(input, MILLISECONDS.toSeconds(currentTimeMs)), - CheckAnswer(), - AddData(input, MILLISECONDS.toSeconds(currentTimeMs)), - CheckAnswer(), - assertEventStats { e => - assert(timestampFormat.parse(e.get("max")).getTime === - SECONDS.toMillis(MILLISECONDS.toSeconds((currentTimeMs)))) - val watermarkTime = timestampFormat.parse(e.get("watermark")) - val monthDiff = monthsSinceEpoch(currentTime) - monthsSinceEpoch(watermarkTime) - // monthsSinceEpoch is like `math.floor(num)`, so monthDiff has two possible values. - assert(monthDiff === 29 || monthDiff === 30, - s"currentTime: $currentTime, watermarkTime: $watermarkTime") - } - ) - } + ) } test("recovery") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val df = 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]) + val inputData = MemoryStream[Int] + val df = 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]) - testStream(df)( - AddData(inputData, 10, 11, 12, 13, 14, 15), - CheckAnswer(), - AddData(inputData, 25), // Advance watermark to 15 seconds - CheckAnswer((10, 5)), - StopStream, - AssertOnQuery { q => // purge commit and clear the sink - val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) - q.commitLog.purge(commit) - q.sink.asInstanceOf[MemorySink].clear() - true - }, - StartStream(), - AddData(inputData, 10, 27, 30), // Advance watermark to 20 seconds, 10 should be ignored - CheckAnswer((15, 1)), - StopStream, - StartStream(), - AddData(inputData, 17), // Watermark should still be 20 seconds, 17 should be ignored - CheckAnswer((15, 1)), - AddData(inputData, 40), // Advance watermark to 30 seconds, emit first data 25 - CheckNewAnswer((25, 2)) - ) - } + testStream(df)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckAnswer(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckAnswer((10, 5)), + StopStream, + AssertOnQuery { q => // purge commit and clear the sink + val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) + q.commitLog.purge(commit) + q.sink.asInstanceOf[MemorySink].clear() + true + }, + StartStream(), + AddData(inputData, 10, 27, 30), // Advance watermark to 20 seconds, 10 should be ignored + CheckAnswer((15, 1)), + StopStream, + StartStream(), + AddData(inputData, 17), // Watermark should still be 20 seconds, 17 should be ignored + CheckAnswer((15, 1)), + AddData(inputData, 40), // Advance watermark to 30 seconds, emit first data 25 + CheckNewAnswer((25, 2)) + ) } test("watermark with 2 streams") { import org.apache.spark.sql.functions.sum - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val first = MemoryStream[Int] + val first = MemoryStream[Int] - val firstDf = first.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "10 seconds") - .select('value) + val firstDf = first.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .select('value) - val second = MemoryStream[Int] + val second = MemoryStream[Int] - val secondDf = second.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "5 seconds") - .select('value) - - withTempDir { checkpointDir => - val unionWriter = firstDf.union(secondDf).agg(sum('value)) - .writeStream - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .format("memory") - .outputMode("complete") - .queryName("test") - - val union = unionWriter.start() - - def getWatermarkAfterData( - firstData: Seq[Int] = Seq.empty, - secondData: Seq[Int] = Seq.empty, - query: StreamingQuery = union): Long = { - if (firstData.nonEmpty) first.addData(firstData) - if (secondData.nonEmpty) second.addData(secondData) - query.processAllAvailable() - // add a dummy batch so lastExecution has the new watermark - first.addData(0) - query.processAllAvailable() - // get last watermark - val lastExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution - lastExecution.offsetSeqMetadata.batchWatermarkMs - } - - // Global watermark starts at 0 until we get data from both sides - assert(getWatermarkAfterData(firstData = Seq(11)) == 0) - assert(getWatermarkAfterData(secondData = Seq(6)) == 1000) - // Global watermark stays at left watermark 1 when right watermark moves to 2 - assert(getWatermarkAfterData(secondData = Seq(8)) == 1000) - // Global watermark switches to right side value 2 when left watermark goes higher - assert(getWatermarkAfterData(firstData = Seq(21)) == 3000) - // Global watermark goes back to left - assert(getWatermarkAfterData(secondData = Seq(17, 28, 39)) == 11000) - // Global watermark stays on left as long as it's below right - assert(getWatermarkAfterData(firstData = Seq(31)) == 21000) - assert(getWatermarkAfterData(firstData = Seq(41)) == 31000) - // Global watermark switches back to right again - assert(getWatermarkAfterData(firstData = Seq(51)) == 34000) - - // Global watermark is updated correctly with simultaneous data from both sides - assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(100)) == 90000) - assert(getWatermarkAfterData(firstData = Seq(120), secondData = Seq(110)) == 105000) - assert(getWatermarkAfterData(firstData = Seq(130), secondData = Seq(125)) == 120000) - - // Global watermark doesn't decrement with simultaneous data - assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(100)) == 120000) - assert(getWatermarkAfterData(firstData = Seq(140), secondData = Seq(100)) == 120000) - assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(135)) == 130000) - - // Global watermark recovers after restart, but left side watermark ahead of it does not. - assert(getWatermarkAfterData(firstData = Seq(200), secondData = Seq(190)) == 185000) - union.stop() - val union2 = unionWriter.start() - assert(getWatermarkAfterData(query = union2) == 185000) - // Even though the left side was ahead of 185000 in the last execution, the watermark won't - // increment until it gets past it in this execution. - assert(getWatermarkAfterData(secondData = Seq(200), query = union2) == 185000) - assert(getWatermarkAfterData(firstData = Seq(200), query = union2) == 190000) + val secondDf = second.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "5 seconds") + .select('value) + + withTempDir { checkpointDir => + val unionWriter = firstDf.union(secondDf).agg(sum('value)) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("memory") + .outputMode("complete") + .queryName("test") + + val union = unionWriter.start() + + def getWatermarkAfterData( + firstData: Seq[Int] = Seq.empty, + secondData: Seq[Int] = Seq.empty, + query: StreamingQuery = union): Long = { + if (firstData.nonEmpty) first.addData(firstData) + if (secondData.nonEmpty) second.addData(secondData) + query.processAllAvailable() + // add a dummy batch so lastExecution has the new watermark + first.addData(0) + query.processAllAvailable() + // get last watermark + val lastExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution + lastExecution.offsetSeqMetadata.batchWatermarkMs } + + // Global watermark starts at 0 until we get data from both sides + assert(getWatermarkAfterData(firstData = Seq(11)) == 0) + assert(getWatermarkAfterData(secondData = Seq(6)) == 1000) + // Global watermark stays at left watermark 1 when right watermark moves to 2 + assert(getWatermarkAfterData(secondData = Seq(8)) == 1000) + // Global watermark switches to right side value 2 when left watermark goes higher + assert(getWatermarkAfterData(firstData = Seq(21)) == 3000) + // Global watermark goes back to left + assert(getWatermarkAfterData(secondData = Seq(17, 28, 39)) == 11000) + // Global watermark stays on left as long as it's below right + assert(getWatermarkAfterData(firstData = Seq(31)) == 21000) + assert(getWatermarkAfterData(firstData = Seq(41)) == 31000) + // Global watermark switches back to right again + assert(getWatermarkAfterData(firstData = Seq(51)) == 34000) + + // Global watermark is updated correctly with simultaneous data from both sides + assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(100)) == 90000) + assert(getWatermarkAfterData(firstData = Seq(120), secondData = Seq(110)) == 105000) + assert(getWatermarkAfterData(firstData = Seq(130), secondData = Seq(125)) == 120000) + + // Global watermark doesn't decrement with simultaneous data + assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(100)) == 120000) + assert(getWatermarkAfterData(firstData = Seq(140), secondData = Seq(100)) == 120000) + assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(135)) == 130000) + + // Global watermark recovers after restart, but left side watermark ahead of it does not. + assert(getWatermarkAfterData(firstData = Seq(200), secondData = Seq(190)) == 185000) + union.stop() + val union2 = unionWriter.start() + assert(getWatermarkAfterData(query = union2) == 185000) + // Even though the left side was ahead of 185000 in the last execution, the watermark won't + // increment until it gets past it in this execution. + assert(getWatermarkAfterData(secondData = Seq(200), query = union2) == 185000) + assert(getWatermarkAfterData(firstData = Seq(200), query = union2) == 190000) } } test("complete mode") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int] - val windowedAggregation = inputData.toDF() + val windowedAggregation = 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]) - // No eviction when asked to compute complete results. - testStream(windowedAggregation, OutputMode.Complete)( - AddData(inputData, 10, 11, 12), - CheckAnswer((10, 3)), - AddData(inputData, 25), - CheckAnswer((10, 3), (25, 1)), - AddData(inputData, 25), - CheckAnswer((10, 3), (25, 2)), - AddData(inputData, 10), - CheckAnswer((10, 4), (25, 2)), - AddData(inputData, 25), - CheckAnswer((10, 4), (25, 3)) - ) - } + // No eviction when asked to compute complete results. + testStream(windowedAggregation, OutputMode.Complete)( + AddData(inputData, 10, 11, 12), + CheckAnswer((10, 3)), + AddData(inputData, 25), + CheckAnswer((10, 3), (25, 1)), + AddData(inputData, 25), + CheckAnswer((10, 3), (25, 2)), + AddData(inputData, 10), + CheckAnswer((10, 4), (25, 2)), + AddData(inputData, 25), + CheckAnswer((10, 4), (25, 3)) + ) } test("group by on raw timestamp") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] + val inputData = MemoryStream[Int] - val windowedAggregation = inputData.toDF() + val windowedAggregation = inputData.toDF() .withColumn("eventTime", $"value".cast("timestamp")) .withWatermark("eventTime", "10 seconds") .groupBy($"eventTime") .agg(count("*") as 'count) .select($"eventTime".cast("long").as[Long], $"count".as[Long]) - testStream(windowedAggregation)( - AddData(inputData, 10), - CheckAnswer(), - AddData(inputData, 25), // Advance watermark to 15 seconds - CheckAnswer((10, 1)) - ) - } + testStream(windowedAggregation)( + AddData(inputData, 10), + CheckAnswer(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckAnswer((10, 1)) + ) } test("delay threshold should not be negative.") { @@ -564,30 +549,26 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } test("the new watermark should override the old one") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val df = MemoryStream[(Long, Long)].toDF() - .withColumn("first", $"_1".cast("timestamp")) - .withColumn("second", $"_2".cast("timestamp")) - .withWatermark("first", "1 minute") - .withWatermark("second", "2 minutes") - - val eventTimeColumns = df.logicalPlan.output - .filter(_.metadata.contains(EventTimeWatermark.delayKey)) - assert(eventTimeColumns.size === 1) - assert(eventTimeColumns(0).name === "second") - } + val df = MemoryStream[(Long, Long)].toDF() + .withColumn("first", $"_1".cast("timestamp")) + .withColumn("second", $"_2".cast("timestamp")) + .withWatermark("first", "1 minute") + .withWatermark("second", "2 minutes") + + val eventTimeColumns = df.logicalPlan.output + .filter(_.metadata.contains(EventTimeWatermark.delayKey)) + assert(eventTimeColumns.size === 1) + assert(eventTimeColumns(0).name === "second") } test("EventTime watermark should be ignored in batch query.") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val df = testData - .withColumn("eventTime", $"key".cast("timestamp")) - .withWatermark("eventTime", "1 minute") - .select("eventTime") - .as[Long] - - checkDataset[Long](df, 1L to 100L: _*) - } + val df = testData + .withColumn("eventTime", $"key".cast("timestamp")) + .withWatermark("eventTime", "1 minute") + .select("eventTime") + .as[Long] + + checkDataset[Long](df, 1L to 100L: _*) } test("SPARK-21565: watermark operator accepts attributes from replacement") { @@ -619,57 +600,53 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } test("SPARK-27340 Alias on TimeWindow expression cause watermark metadata lost") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + val inputData = MemoryStream[Int] + val aliasWindow = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .select(window($"eventTime", "5 seconds") as 'aliasWindow) + // Check the eventTime metadata is kept in the top level alias. + assert(aliasWindow.logicalPlan.output.exists( + _.metadata.contains(EventTimeWatermark.delayKey))) + + val windowedAggregation = aliasWindow + .groupBy('aliasWindow) + .agg(count("*") as 'count) + .select($"aliasWindow".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(windowedAggregation)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckNewAnswer(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckNewAnswer((10, 5)), + assertNumStateRows(2), + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckNewAnswer(), + assertNumStateRows(2) + ) + } + + test("test no-data flag") { + val flagKey = SQLConf.STREAMING_NO_DATA_MICRO_BATCHES_ENABLED.key + + def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] - val aliasWindow = inputData.toDF() + val windowedAggregation = inputData.toDF() .withColumn("eventTime", $"value".cast("timestamp")) .withWatermark("eventTime", "10 seconds") - .select(window($"eventTime", "5 seconds") as 'aliasWindow) - // Check the eventTime metadata is kept in the top level alias. - assert(aliasWindow.logicalPlan.output.exists( - _.metadata.contains(EventTimeWatermark.delayKey))) - - val windowedAggregation = aliasWindow - .groupBy('aliasWindow) + .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) - .select($"aliasWindow".getField("start").cast("long").as[Long], $"count".as[Long]) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) testStream(windowedAggregation)( + StartStream(additionalConfs = Map(flagKey -> flag.toString)), AddData(inputData, 10, 11, 12, 13, 14, 15), CheckNewAnswer(), AddData(inputData, 25), // Advance watermark to 15 seconds - CheckNewAnswer((10, 5)), - assertNumStateRows(2), - AddData(inputData, 10), // Should not emit anything as data less than watermark - CheckNewAnswer(), - assertNumStateRows(2) + // Check if there is new answer if flag is set, no new answer otherwise + if (flag) CheckNewAnswer((10, 5)) else CheckNewAnswer() ) } - } - - test("test no-data flag") { - val flagKey = SQLConf.STREAMING_NO_DATA_MICRO_BATCHES_ENABLED.key - - def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val windowedAggregation = 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]) - - testStream(windowedAggregation)( - StartStream(additionalConfs = Map(flagKey -> flag.toString)), - AddData(inputData, 10, 11, 12, 13, 14, 15), - CheckNewAnswer(), - AddData(inputData, 25), // Advance watermark to 15 seconds - // Check if there is new answer if flag is set, no new answer otherwise - if (flag) CheckNewAnswer((10, 5)) else CheckNewAnswer() - ) - } - } testWithFlag(true) testWithFlag(false) @@ -679,8 +656,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val input1 = MemoryStream[Int] val input2 = MemoryStream[Int] - withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "max", - SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "max") { testStream(dfWithMultipleWatermarks(input1, input2))( MultiAddData(input1, 20)(input2, 30), CheckLastBatch(20, 30), @@ -702,8 +678,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val input1 = MemoryStream[Int] val input2 = MemoryStream[Int] - withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "min", - SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "min") { testStream(dfWithMultipleWatermarks(input1, input2))( MultiAddData(input1, 20)(input2, 30), CheckLastBatch(20, 30), @@ -726,8 +701,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val input2 = MemoryStream[Int] val checkpointDir = Utils.createTempDir().getCanonicalFile - withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "max", - SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "max") { testStream(dfWithMultipleWatermarks(input1, input2))( StartStream(checkpointLocation = checkpointDir.getAbsolutePath), MultiAddData(input1, 20)(input2, 30), @@ -736,8 +710,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche ) } - withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "min", - SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "min") { testStream(dfWithMultipleWatermarks(input1, input2))( StartStream(checkpointLocation = checkpointDir.getAbsolutePath), checkWatermark(input1, 15), // watermark recovered correctly @@ -752,37 +725,32 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } test("MultipleWatermarkPolicy: recovery from Spark ver 2.3.1 checkpoints ensures min policy") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val input1 = MemoryStream[Int] - val input2 = MemoryStream[Int] - - val resourceUri = this.getClass.getResource( - "/structured-streaming/checkpoint-version-2.3.1-for-multi-watermark-policy/").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) - - input1.addData(20) - input2.addData(30) - input1.addData(10) - - withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "max") { - testStream(dfWithMultipleWatermarks(input1, input2))( - StartStream(checkpointLocation = checkpointDir.getAbsolutePath), - Execute { - _.processAllAvailable() - }, - MultiAddData(input1, 120)(input2, 130), - CheckLastBatch(120, 130), - // should calculate 'min' even if session conf has 'max' policy - checkWatermark(input2, 110), - AddData(input2, 150), - CheckLastBatch(150), - checkWatermark(input2, 110) - ) - } + val input1 = MemoryStream[Int] + val input2 = MemoryStream[Int] + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-2.3.1-for-multi-watermark-policy/").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) + + input1.addData(20) + input2.addData(30) + input1.addData(10) + + withSQLConf(SQLConf.STREAMING_MULTIPLE_WATERMARK_POLICY.key -> "max") { + testStream(dfWithMultipleWatermarks(input1, input2))( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath), + Execute { _.processAllAvailable() }, + MultiAddData(input1, 120)(input2, 130), + CheckLastBatch(120, 130), + checkWatermark(input2, 110), // should calculate 'min' even if session conf has 'max' policy + AddData(input2, 150), + CheckLastBatch(150), + checkWatermark(input2, 110) + ) } } 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 e7af44a02d67..bdd04b273b6f 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 @@ -46,11 +46,14 @@ abstract class FileStreamSinkSuite extends StreamTest { override def beforeAll(): Unit = { super.beforeAll() spark.sessionState.conf.setConf(SQLConf.ORC_IMPLEMENTATION, "native") + spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) } override def afterAll(): Unit = { try { spark.sessionState.conf.unsetConf(SQLConf.ORC_IMPLEMENTATION) + spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) } finally { super.afterAll() } @@ -207,61 +210,59 @@ abstract class FileStreamSinkSuite extends StreamTest { // Since FileStreamSink currently only supports append mode, we will test FileStreamSink // with aggregations using event time windows and watermark, which allows // aggregation + append mode. - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Long] - val inputDF = inputData.toDF.toDF("time") - val outputDf = inputDF - .selectExpr("CAST(time AS timestamp) AS timestamp") - .withWatermark("timestamp", "10 seconds") - .groupBy(window($"timestamp", "5 seconds")) - .count() - .select("window.start", "window.end", "count") + val inputData = MemoryStream[Long] + val inputDF = inputData.toDF.toDF("time") + val outputDf = inputDF + .selectExpr("CAST(time AS timestamp) AS timestamp") + .withWatermark("timestamp", "10 seconds") + .groupBy(window($"timestamp", "5 seconds")) + .count() + .select("window.start", "window.end", "count") - val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath - val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath - var query: StreamingQuery = null + var query: StreamingQuery = null - try { - query = - outputDf.writeStream - .option("checkpointLocation", checkpointDir) - .format("parquet") - .start(outputDir) + try { + query = + outputDf.writeStream + .option("checkpointLocation", checkpointDir) + .format("parquet") + .start(outputDir) - def addTimestamp(timestampInSecs: Int*): Unit = { - inputData.addData(timestampInSecs.map(_ * 1L): _*) - failAfter(streamingTimeout) { - query.processAllAvailable() - } + def addTimestamp(timestampInSecs: Int*): Unit = { + inputData.addData(timestampInSecs.map(_ * 1L): _*) + failAfter(streamingTimeout) { + query.processAllAvailable() } + } - def check(expectedResult: ((Long, Long), Long)*): Unit = { - val outputDf = spark.read.parquet(outputDir) - .selectExpr( - "CAST(start as BIGINT) AS start", - "CAST(end as BIGINT) AS end", - "count") - .orderBy("start") // sort the DataFrame in order to compare with the expected one. - checkDataset( - outputDf.as[(Long, Long, Long)], - expectedResult.map(x => (x._1._1, x._1._2, x._2)): _*) - } + def check(expectedResult: ((Long, Long), Long)*): Unit = { + val outputDf = spark.read.parquet(outputDir) + .selectExpr( + "CAST(start as BIGINT) AS start", + "CAST(end as BIGINT) AS end", + "count") + .orderBy("start") // sort the DataFrame in order to compare with the expected one. + checkDataset( + outputDf.as[(Long, Long, Long)], + expectedResult.map(x => (x._1._1, x._1._2, x._2)): _*) + } - addTimestamp(100) // watermark = None before this, watermark = 100 - 10 = 90 after this - check() // nothing emitted yet + addTimestamp(100) // watermark = None before this, watermark = 100 - 10 = 90 after this + check() // nothing emitted yet - addTimestamp(104, 123) // watermark = 90 before this, watermark = 123 - 10 = 113 after this - check((100L, 105L) -> 2L) // no-data-batch emits results on 100-105, + addTimestamp(104, 123) // watermark = 90 before this, watermark = 123 - 10 = 113 after this + check((100L, 105L) -> 2L) // no-data-batch emits results on 100-105, - addTimestamp(140) // wm = 113 before this, emit results on 100-105, wm = 130 after this - check((100L, 105L) -> 2L, (120L, 125L) -> 1L) // no-data-batch emits results on 120-125 + addTimestamp(140) // wm = 113 before this, emit results on 100-105, wm = 130 after this + check((100L, 105L) -> 2L, (120L, 125L) -> 1L) // no-data-batch emits results on 120-125 - } finally { - if (query != null) { - query.stop() - } + } finally { + if (query != null) { + query.stop() } } } 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 2b18a23f951d..f39b2c528895 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 @@ -21,7 +21,7 @@ import java.io.File import java.sql.Date import org.apache.commons.io.FileUtils -import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfter import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkException @@ -45,13 +45,22 @@ case class RunningCount(count: Long) case class Result(key: Long, count: Int) -class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { +class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with BeforeAndAfter { import testImplicits._ import GroupStateImpl._ import GroupStateTimeout._ import FlatMapGroupsWithStateSuite._ + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } + test("GroupState - get, exists, update, remove") { var state: GroupStateImpl[String] = null @@ -806,138 +815,123 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { // Function to maintain the max event time as state and set the timeout timestamp based on the // current max event time seen. It returns the max event time in the state, or -1 if the state // was removed by timeout. - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => { - assertCanGetProcessingTime { - state.getCurrentProcessingTimeMs() >= 0 - } - assertCanGetWatermark { - state.getCurrentWatermarkMs() >= -1 - } + val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => { + assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 } + assertCanGetWatermark { state.getCurrentWatermarkMs() >= -1 } - val timeoutDelaySec = 5 - if (state.hasTimedOut) { - state.remove() - Iterator((key, -1)) - } else { - val valuesSeq = values.toSeq - val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) - val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec - state.update(maxEventTimeSec) - state.setTimeoutTimestamp(timeoutTimestampSec * 1000) - Iterator((key, maxEventTimeSec.toInt)) - } + val timeoutDelaySec = 5 + if (state.hasTimedOut) { + state.remove() + Iterator((key, -1)) + } else { + val valuesSeq = values.toSeq + val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) + val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec + state.update(maxEventTimeSec) + state.setTimeoutTimestamp(timeoutTimestampSec * 1000) + Iterator((key, maxEventTimeSec.toInt)) } - val inputData = MemoryStream[(String, Int)] - val result = - inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) - .withWatermark("eventTime", "10 seconds") - .as[(String, Long)] - .groupByKey(_._1) - .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) + } + val inputData = MemoryStream[(String, Int)] + val result = + inputData.toDS + .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .withWatermark("eventTime", "10 seconds") + .as[(String, Long)] + .groupByKey(_._1) + .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) - testStream(result, Update)( - StartStream(), + testStream(result, Update)( + StartStream(), - AddData(inputData, ("a", 11), ("a", 13), ("a", 15)), - // Max event time = 15. Timeout timestamp for "a" = 15 + 5 = 20. Watermark = 15 - 10 = 5. - CheckNewAnswer(("a", 15)), // Output = max event time of a + AddData(inputData, ("a", 11), ("a", 13), ("a", 15)), + // Max event time = 15. Timeout timestamp for "a" = 15 + 5 = 20. Watermark = 15 - 10 = 5. + CheckNewAnswer(("a", 15)), // Output = max event time of a - AddData(inputData, ("a", 4)), // Add data older than watermark for "a" - CheckNewAnswer(), // No output as data should get filtered by watermark + AddData(inputData, ("a", 4)), // Add data older than watermark for "a" + CheckNewAnswer(), // No output as data should get filtered by watermark - AddData(inputData, ("a", 10)), // Add data newer than watermark for "a" - CheckNewAnswer(("a", 15)), // Max event time is still the same - // Timeout timestamp for "a" is still 20 as max event time for "a" is still 15. - // Watermark is still 5 as max event time for all data is still 15. + AddData(inputData, ("a", 10)), // Add data newer than watermark for "a" + CheckNewAnswer(("a", 15)), // Max event time is still the same + // Timeout timestamp for "a" is still 20 as max event time for "a" is still 15. + // Watermark is still 5 as max event time for all data is still 15. - AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a" - // Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20. - CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1 - ) - } + AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a" + // Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20. + CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1 + ) } test("flatMapGroupsWithState - uses state format version 2 by default") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { - val count = state.getOption.map(_.count).getOrElse(0L) + values.size - state.update(RunningCount(count)) - Iterator((key, count.toString)) - } + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + Iterator((key, count.toString)) + } - val inputData = MemoryStream[String] - val result = inputData.toDS() + val inputData = MemoryStream[String] + val result = inputData.toDS() .groupByKey(x => x) .flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc) - testStream(result, Update)( - AddData(inputData, "a"), - CheckNewAnswer(("a", "1")), - Execute { query => - // Verify state format = 2 - val f = query.lastExecution.executedPlan.collect - { case f: FlatMapGroupsWithStateExec => f } - assert(f.size == 1) - assert(f.head.stateFormatVersion == 2) - } - ) - } + testStream(result, Update)( + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + Execute { query => + // Verify state format = 2 + val f = query.lastExecution.executedPlan.collect { case f: FlatMapGroupsWithStateExec => f } + assert(f.size == 1) + assert(f.head.stateFormatVersion == 2) + } + ) } test("flatMapGroupsWithState - recovery from checkpoint uses state format version 1") { // Function to maintain the max event time as state and set the timeout timestamp based on the // current max event time seen. It returns the max event time in the state, or -1 if the state // was removed by timeout. - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => { - assertCanGetProcessingTime { - state.getCurrentProcessingTimeMs() >= 0 - } - assertCanGetWatermark { - state.getCurrentWatermarkMs() >= -1 - } + val stateFunc = (key: String, values: Iterator[(String, Long)], state: GroupState[Long]) => { + assertCanGetProcessingTime { state.getCurrentProcessingTimeMs() >= 0 } + assertCanGetWatermark { state.getCurrentWatermarkMs() >= -1 } - val timeoutDelaySec = 5 - if (state.hasTimedOut) { - state.remove() - Iterator((key, -1)) - } else { - val valuesSeq = values.toSeq - val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) - val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec - state.update(maxEventTimeSec) - state.setTimeoutTimestamp(timeoutTimestampSec * 1000) - Iterator((key, maxEventTimeSec.toInt)) - } + val timeoutDelaySec = 5 + if (state.hasTimedOut) { + state.remove() + Iterator((key, -1)) + } else { + val valuesSeq = values.toSeq + val maxEventTimeSec = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) + val timeoutTimestampSec = maxEventTimeSec + timeoutDelaySec + state.update(maxEventTimeSec) + state.setTimeoutTimestamp(timeoutTimestampSec * 1000) + Iterator((key, maxEventTimeSec.toInt)) } - val inputData = MemoryStream[(String, Int)] - val result = - inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) - .withWatermark("eventTime", "10 seconds") - .as[(String, Long)] - .groupByKey(_._1) - .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) + } + val inputData = MemoryStream[(String, Int)] + val result = + inputData.toDS + .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .withWatermark("eventTime", "10 seconds") + .as[(String, Long)] + .groupByKey(_._1) + .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) - val resourceUri = this.getClass.getResource( - "/structured-streaming/checkpoint-version-2.3.1-flatMapGroupsWithState-state-format-1/") - .toURI + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-2.3.1-flatMapGroupsWithState-state-format-1/").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) + 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) - inputData.addData(("a", 11), ("a", 13), ("a", 15)) - inputData.addData(("a", 4)) + inputData.addData(("a", 11), ("a", 13), ("a", 15)) + inputData.addData(("a", 4)) - testStream(result, Update)( - StartStream( - checkpointLocation = checkpointDir.getAbsolutePath, - additionalConfs = Map(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> "2")), - /* + testStream(result, Update)( + StartStream( + checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = Map(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> "2")), + /* Note: The checkpoint was generated using the following input in Spark version 2.3.1 AddData(inputData, ("a", 11), ("a", 13), ("a", 15)), @@ -948,24 +942,22 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { CheckNewAnswer(), // No output as data should get filtered by watermark */ - AddData(inputData, ("a", 10)), // Add data newer than watermark for "a" - CheckNewAnswer(("a", 15)), // Max event time is still the same - // Timeout timestamp for "a" is still 20 as max event time for "a" is still 15. - // Watermark is still 5 as max event time for all data is still 15. - - Execute { query => - // Verify state format = 1 - val f = query.lastExecution.executedPlan.collect { case f: - FlatMapGroupsWithStateExec => f } - assert(f.size == 1) - assert(f.head.stateFormatVersion == 1) - }, + AddData(inputData, ("a", 10)), // Add data newer than watermark for "a" + CheckNewAnswer(("a", 15)), // Max event time is still the same + // Timeout timestamp for "a" is still 20 as max event time for "a" is still 15. + // Watermark is still 5 as max event time for all data is still 15. - AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a" - // Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20. - CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1 - ) - } + Execute { query => + // Verify state format = 1 + val f = query.lastExecution.executedPlan.collect { case f: FlatMapGroupsWithStateExec => f } + assert(f.size == 1) + assert(f.head.stateFormatVersion == 1) + }, + + AddData(inputData, ("b", 31)), // Add data newer than watermark for "b", not "a" + // Watermark = 31 - 10 = 21, so "a" should be timed out as timeout timestamp for "a" is 20. + CheckNewAnswer(("a", -1), ("b", 31)) // State for "a" should timeout and emit -1 + ) } @@ -1111,37 +1103,35 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { test("SPARK-20714: watermark does not fail query when timeout = " + timeoutConf) { // Function to maintain running count up to 2, and then remove the count // Returns the data and the count (-1 if count reached beyond 2 and state was just removed) - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val stateFunc = - (key: String, values: Iterator[(String, Long)], state: GroupState[RunningCount]) => { - if (state.hasTimedOut) { - state.remove() - Iterator((key, "-1")) - } else { - val count = state.getOption.map(_.count).getOrElse(0L) + values.size - state.update(RunningCount(count)) - state.setTimeoutDuration("10 seconds") - Iterator((key, count.toString)) - } - } - - val clock = new StreamManualClock - val inputData = MemoryStream[(String, Long)] - val result = - inputData.toDF().toDF("key", "time") - .selectExpr("key", "cast(time as timestamp) as timestamp") - .withWatermark("timestamp", "10 second") - .as[(String, Long)] - .groupByKey(x => x._1) - .flatMapGroupsWithState(Update, ProcessingTimeTimeout)(stateFunc) - - testStream(result, Update)( - StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), - AddData(inputData, ("a", 1L)), - AdvanceManualClock(1 * 1000), - CheckNewAnswer(("a", "1")) - ) + val stateFunc = + (key: String, values: Iterator[(String, Long)], state: GroupState[RunningCount]) => { + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, Long)] + val result = + inputData.toDF().toDF("key", "time") + .selectExpr("key", "cast(time as timestamp) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, Long)] + .groupByKey(x => x._1) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout)(stateFunc) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputData, ("a", 1L)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("a", "1")) + ) } } testWithTimeout(NoTimeout) 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 dd38e27ea430..9d1e65f777c6 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 @@ -23,7 +23,7 @@ import java.util.{Locale, TimeZone} import scala.collection.mutable import org.apache.commons.io.FileUtils -import org.scalatest.Assertions +import org.scalatest.{Assertions, BeforeAndAfter} import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.rdd.BlockRDD @@ -48,10 +48,19 @@ object FailureSingleton { var firstTime = true } -class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { +class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions with BeforeAndAfter{ import testImplicits._ + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } + def executeFuncWithStateVersionSQLConf( stateVersion: Int, confPairs: Seq[(String, String)], @@ -186,133 +195,85 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { } testWithAllStateVersions("state metrics - append mode") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - 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 - } - } - - // Pick the latest progress that actually ran a batch - def lastExecutedBatch: StreamingQueryProgress = { - query.recentProgress.filter(_.durationMs.containsKey("addBatch")).last - } + 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]) - def stateOperatorProgresses: Seq[StateOperatorProgress] = { - lastExecutedBatch.stateOperators + 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 } } - val clock = new StreamManualClock() - - testStream(aggWithWatermark)( - // batchId 0 - AddData(inputData, 15), - StartStream(Trigger.ProcessingTime("interval 1 second"), clock), - CheckAnswer(), // watermark = 0 - AssertOnQuery { - _.stateNodes.size === 1 - }, - AssertOnQuery { - _.stateNodes.head.metrics("numOutputRows").value === 0 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsUpdated === 1 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsTotal === 1 - }, - AssertOnQuery { - _.lastExecutedBatch.sink.numOutputRows == 0 - }, - - // batchId 1 without data - AdvanceManualClock(1000L), // watermark = 5 - Execute { q => // wait for the no data batch to complete - eventually(timeout(streamingTimeout)) { - assert(q.lastProgress.batchId === 1) - } - }, - CheckAnswer(), - AssertOnQuery { - _.stateNodes.head.metrics("numOutputRows").value === 0 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsUpdated === 0 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsTotal === 1 - }, - AssertOnQuery { - _.lastExecutedBatch.sink.numOutputRows == 0 - }, + // Pick the latest progress that actually ran a batch + def lastExecutedBatch: StreamingQueryProgress = { + query.recentProgress.filter(_.durationMs.containsKey("addBatch")).last + } - // batchId 2 with data - AddData(inputData, 10, 12, 14), - AdvanceManualClock(1000L), // watermark = 5 - CheckAnswer(), - AssertOnQuery { - _.stateNodes.head.metrics("numOutputRows").value === 0 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsUpdated === 1 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsTotal === 2 - }, - AssertOnQuery { - _.lastExecutedBatch.sink.numOutputRows == 0 - }, + def stateOperatorProgresses: Seq[StateOperatorProgress] = { + lastExecutedBatch.stateOperators + } + } - // batchId 3 with data - AddData(inputData, 25), - AdvanceManualClock(1000L), // watermark = 5 - CheckAnswer(), - AssertOnQuery { - _.stateNodes.head.metrics("numOutputRows").value === 0 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsUpdated === 1 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsTotal === 3 - }, - AssertOnQuery { - _.lastExecutedBatch.sink.numOutputRows == 0 - }, + val clock = new StreamManualClock() - // batchId 4 without data - AdvanceManualClock(1000L), // watermark = 15 - Execute { q => // wait for the no data batch to complete - eventually(timeout(streamingTimeout)) { - assert(q.lastProgress.batchId === 4) - } - }, - CheckAnswer((10, 3)), - AssertOnQuery { - _.stateNodes.head.metrics("numOutputRows").value === 1 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsUpdated === 0 - }, - AssertOnQuery { - _.stateOperatorProgresses.head.numRowsTotal === 2 - }, - AssertOnQuery { - _.lastExecutedBatch.sink.numOutputRows == 1 - } - ) - } + testStream(aggWithWatermark)( + // batchId 0 + AddData(inputData, 15), + StartStream(Trigger.ProcessingTime("interval 1 second"), clock), + CheckAnswer(), // watermark = 0 + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 1 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 1 without data + AdvanceManualClock(1000L), // watermark = 5 + Execute { q => // wait for the no data batch to complete + eventually(timeout(streamingTimeout)) { assert(q.lastProgress.batchId === 1) } + }, + CheckAnswer(), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 1 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 2 with data + AddData(inputData, 10, 12, 14), + AdvanceManualClock(1000L), // watermark = 5 + CheckAnswer(), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 3 with data + AddData(inputData, 25), + AdvanceManualClock(1000L), // watermark = 5 + CheckAnswer(), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 3 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 4 without data + AdvanceManualClock(1000L), // watermark = 15 + Execute { q => // wait for the no data batch to complete + eventually(timeout(streamingTimeout)) { assert(q.lastProgress.batchId === 4) } + }, + CheckAnswer((10, 3)), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 1 } + ) } testWithAllStateVersions("state metrics - update/complete mode") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index cafef7821a54..a1b389ecddcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfter import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ @@ -26,10 +26,19 @@ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -class StreamingDeduplicationSuite extends StateStoreMetricsTest { +class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAfter{ import testImplicits._ + before { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } + test("deduplicate with all columns") { val inputData = MemoryStream[String] val result = inputData.toDS().dropDuplicates() @@ -84,71 +93,66 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { } test("deduplicate with watermark") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "10 seconds") - .dropDuplicates() - .select($"eventTime".cast("long").as[Long]) + val inputData = MemoryStream[Int] + val result = inputData.toDS() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .dropDuplicates() + .select($"eventTime".cast("long").as[Long]) - testStream(result, Append)( - AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*), - CheckAnswer(10 to 15: _*), - assertNumStateRows(total = 6, updated = 6), + testStream(result, Append)( + AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*), + CheckAnswer(10 to 15: _*), + assertNumStateRows(total = 6, updated = 6), - AddData(inputData, 25), // Advance watermark to 15 secs, no-data-batch drops rows <= 15 - CheckNewAnswer(25), - assertNumStateRows(total = 1, updated = 1), + AddData(inputData, 25), // Advance watermark to 15 secs, no-data-batch drops rows <= 15 + CheckNewAnswer(25), + assertNumStateRows(total = 1, updated = 1), - AddData(inputData, 10), // Should not emit anything as data less than watermark - CheckNewAnswer(), - assertNumStateRows(total = 1, updated = 0), + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckNewAnswer(), + assertNumStateRows(total = 1, updated = 0), - AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25 - CheckNewAnswer(45), - assertNumStateRows(total = 1, updated = 1) - ) - } + AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25 + CheckNewAnswer(45), + assertNumStateRows(total = 1, updated = 1) + ) } test("deduplicate with aggregate - append mode") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val inputData = MemoryStream[Int] - val windowedaggregate = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "10 seconds") - .dropDuplicates() - .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]) + val inputData = MemoryStream[Int] + val windowedaggregate = inputData.toDS() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .dropDuplicates() + .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]) + + testStream(windowedaggregate)( + AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*), + CheckLastBatch(), + // states in aggregate in [10, 14), [15, 20) (2 windows) + // states in deduplicate is 10 to 15 + assertNumStateRows(total = Seq(2L, 6L), updated = Seq(2L, 6L)), - testStream(windowedaggregate)( - AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*), - CheckLastBatch(), - // states in aggregate in [10, 14), [15, 20) (2 windows) - // states in deduplicate is 10 to 15 - assertNumStateRows(total = Seq(2L, 6L), updated = Seq(2L, 6L)), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckLastBatch((10 -> 5)), // 5 items (10 to 14) after deduplicate, emitted with no-data-batch + // states in aggregate in [15, 20) and [25, 30); no-data-batch removed [10, 14) + // states in deduplicate is 25, no-data-batch removed 10 to 14 + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(1L, 1L)), - AddData(inputData, 25), // Advance watermark to 15 seconds - // 5 items (10 to 14) after deduplicate, emitted with no-data-batch - CheckLastBatch((10 -> 5)), - // states in aggregate in [15, 20) and [25, 30); no-data-batch removed [10, 14) - // states in deduplicate is 25, no-data-batch removed 10 to 14 - assertNumStateRows(total = Seq(2L, 1L), updated = Seq(1L, 1L)), - - AddData(inputData, 10), // Should not emit anything as data less than watermark - CheckLastBatch(), - assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L)), - - AddData(inputData, 40), // Advance watermark to 30 seconds - CheckLastBatch((15 -> 1), (25 -> 1)), - // states in aggregate is [40, 45); no-data-batch removed [15, 20) and [25, 30) - // states in deduplicate is 40; no-data-batch removed 25 - assertNumStateRows(total = Seq(1L, 1L), updated = Seq(1L, 1L)) - ) - } + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckLastBatch(), + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L)), + + AddData(inputData, 40), // Advance watermark to 30 seconds + CheckLastBatch((15 -> 1), (25 -> 1)), + // states in aggregate is [40, 45); no-data-batch removed [15, 20) and [25, 30) + // states in deduplicate is 40; no-data-batch removed 25 + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(1L, 1L)) + ) } test("deduplicate with aggregate - update mode") { @@ -232,71 +236,65 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { } test("SPARK-19841: watermarkPredicate should filter based on keys") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val input = MemoryStream[(Int, Int)] - val df = input.toDS.toDF("time", "id") - .withColumn("time", $"time".cast("timestamp")) - .withWatermark("time", "1 second") - .dropDuplicates("id", "time") // Change the column positions - .select($"id") - testStream(df)( - AddData(input, 1 -> 1, 1 -> 1, 1 -> 2), - CheckAnswer(1, 2), - AddData(input, 1 -> 1, 2 -> 3, 2 -> 4), - CheckNewAnswer(3, 4), - AddData(input, 1 -> 0, 1 -> 1, 3 -> 5, 3 -> 6), // Drop (1 -> 0, 1 -> 1) due to watermark - CheckNewAnswer(5, 6), - AddData(input, 1 -> 0, 4 -> 7), // Drop (1 -> 0) due to watermark - CheckNewAnswer(7) - ) - } + val input = MemoryStream[(Int, Int)] + val df = input.toDS.toDF("time", "id") + .withColumn("time", $"time".cast("timestamp")) + .withWatermark("time", "1 second") + .dropDuplicates("id", "time") // Change the column positions + .select($"id") + testStream(df)( + AddData(input, 1 -> 1, 1 -> 1, 1 -> 2), + CheckAnswer(1, 2), + AddData(input, 1 -> 1, 2 -> 3, 2 -> 4), + CheckNewAnswer(3, 4), + AddData(input, 1 -> 0, 1 -> 1, 3 -> 5, 3 -> 6), // Drop (1 -> 0, 1 -> 1) due to watermark + CheckNewAnswer(5, 6), + AddData(input, 1 -> 0, 4 -> 7), // Drop (1 -> 0) due to watermark + CheckNewAnswer(7) + ) } test("SPARK-21546: dropDuplicates should ignore watermark when it's not a key") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val input = MemoryStream[(Int, Int)] - val df = input.toDS.toDF("id", "time") - .withColumn("time", $"time".cast("timestamp")) - .withWatermark("time", "1 second") - .dropDuplicates("id") - .select($"id", $"time".cast("long")) - testStream(df)( - AddData(input, 1 -> 1, 1 -> 2, 2 -> 2), - CheckAnswer(1 -> 1, 2 -> 2) - ) - } + val input = MemoryStream[(Int, Int)] + val df = input.toDS.toDF("id", "time") + .withColumn("time", $"time".cast("timestamp")) + .withWatermark("time", "1 second") + .dropDuplicates("id") + .select($"id", $"time".cast("long")) + testStream(df)( + AddData(input, 1 -> 1, 1 -> 2, 2 -> 2), + CheckAnswer(1 -> 1, 2 -> 2) + ) } test("test no-data flag") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - val flagKey = SQLConf.STREAMING_NO_DATA_MICRO_BATCHES_ENABLED.key - - def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { - val inputData = MemoryStream[Int] - val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) - .withWatermark("eventTime", "10 seconds") - .dropDuplicates() - .select($"eventTime".cast("long").as[Long]) - - testStream(result, Append)( - StartStream(additionalConfs = Map(flagKey -> flag.toString)), - AddData(inputData, 10, 11, 12, 13, 14, 15), - CheckAnswer(10, 11, 12, 13, 14, 15), - assertNumStateRows(total = 6, updated = 6), - - AddData(inputData, 25), // Advance watermark to 15 seconds - CheckNewAnswer(25), - { // State should have been cleaned if flag is set, otherwise should not have been cleaned - if (flag) assertNumStateRows(total = 1, updated = 1) - else assertNumStateRows(total = 7, updated = 1) - }, - AssertOnQuery(q => q.lastProgress.sink.numOutputRows == 0L) - ) - } + val flagKey = SQLConf.STREAMING_NO_DATA_MICRO_BATCHES_ENABLED.key - testWithFlag(true) - testWithFlag(false) + def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { + val inputData = MemoryStream[Int] + val result = inputData.toDS() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .dropDuplicates() + .select($"eventTime".cast("long").as[Long]) + + testStream(result, Append)( + StartStream(additionalConfs = Map(flagKey -> flag.toString)), + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckAnswer(10, 11, 12, 13, 14, 15), + assertNumStateRows(total = 6, updated = 6), + + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckNewAnswer(25), + { // State should have been cleaned if flag is set, otherwise should not have been cleaned + if (flag) assertNumStateRows(total = 1, updated = 1) + else assertNumStateRows(total = 7, updated = 1) + }, + AssertOnQuery(q => q.lastProgress.sink.numOutputRows == 0L) + ) } + + testWithFlag(true) + testWithFlag(false) } } 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 b0e270849152..447618a9587e 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 @@ -52,6 +52,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled private val originalLegacyAllowCastNumericToTimestamp = TestHive.conf.legacyAllowCastNumericToTimestamp + def spark: SparkSession = sparkSession override def beforeAll(): Unit = { 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 af50ac1ff2f7..2ca309408570 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 @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.serde2.{AbstractSerDe, SerDeStats} import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.{LongWritable, Writable} +import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.plans.logical.Project @@ -50,11 +51,20 @@ case class ListStringCaseClass(l: Seq[String]) /** * A test suite for Hive custom UDFs. */ -class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { +class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfter{ import spark.udf import spark.implicits._ + before { + spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } + test("spark sql udf test that returns a struct") { udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) assert(sql( @@ -431,7 +441,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("Hive UDF in group by") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { +// withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { withTempView("tab1") { Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") @@ -441,7 +451,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { assert(count == 1) } } - } +// } test("SPARK-11522 select input_file_name from non-parquet table") { From 38f0e76751de25f0c0c053f261d3347bc461252e Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 30 May 2020 22:08:55 +0800 Subject: [PATCH 21/75] fix test case error --- .../sql-tests/results/ansi/datetime.sql.out | 16 ++--- .../sql-tests/results/datetime-legacy.sql.out | 16 ++--- .../sql-tests/results/datetime.sql.out | 17 +++-- .../sql-tests/results/postgreSQL/text.sql.out | 12 ++-- .../sql-tests/results/udf/udf-window.sql.out | 28 +++++--- .../sql-tests/results/window.sql.out | 68 ++++++++++++++----- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 + .../FlatMapGroupsWithStateSuite.scala | 2 +- .../StreamingDeduplicationSuite.scala | 2 +- .../sql/hive/execution/HiveUDFSuite.scala | 18 +++-- .../sql/hive/execution/SQLQuerySuite.scala | 47 +++++++------ 11 files changed, 137 insertions(+), 90 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index e254a3a6471d..5857a0ac90c7 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -79,10 +79,9 @@ select current_date = current_date(), current_timestamp = current_timestamp() -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 +NULL 2016-12-31 2016-12-31 -- !query @@ -812,18 +811,17 @@ NULL -- !query select to_date("16", "dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +1970-01-16 + -- !query select to_date("02-29", "MM-dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 85a78e3307a1..8a726efafad8 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -73,10 +73,9 @@ true true -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 +NULL 2016-12-31 2016-12-31 -- !query @@ -784,18 +783,17 @@ NULL -- !query select to_date("16", "dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +1970-01-16 + -- !query select to_date("02-29", "MM-dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +NULL -- !query 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 dc9c33cf3159..7cacaec42c81 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 @@ -73,10 +73,10 @@ true true -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('2016-12-31', 'yyyy-MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 45 +NULL 2016-12-31 2016-12-31 + -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') @@ -783,18 +783,17 @@ NULL -- !query select to_date("16", "dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('16', 'dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +1970-01-16 + -- !query select to_date("02-29", "MM-dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('02-29', 'MM-dd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 7 +NULL -- !query 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 16367a37a697..811e7d6e4ca6 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 @@ -99,10 +99,9 @@ one -- !query select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 41 +123hellotruefalse2010-03-09 -- !query @@ -116,10 +115,11 @@ one -- !query select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(unix_timestamp('20100309', 'yyyyMMdd') AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 48 +1#x#x#hello#true#false#x-03-09 + + -- !query select concat_ws(',',10,20,null,30) -- !query schema 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 136f489ffb29..a915c1bd6c71 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 @@ -17,8 +17,6 @@ AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot cast int to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 2 pos 37 @@ -26,20 +24,34 @@ cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot 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 schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 11 +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 1 +1 a 1 +2 a 1 +1 b 1 +2 b 1 +3 b 1 -- !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 schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 11 +NULL NULL 3 +3 NULL 3 +NULL a 1 +1 a 2 +1 a 4 +2 a 4 +1 b 3 +2 b 6 +3 b 6 -- !query 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 f6495f1dc8df..625088f90ced 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 @@ -17,28 +17,41 @@ AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1501545600 AS TIMESTAMP)' due to data type mismatch: cannot cast int to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommand using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 2 pos 37 + -- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData ORDER BY cate, val -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 1 +1 a 1 +2 a 1 +1 b 1 +2 b 1 +3 b 1 -- !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 schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +NULL NULL 3 +3 NULL 3 +NULL a 1 +1 a 2 +1 a 4 +2 a 4 +1 b 3 +2 b 6 +3 b 6 -- !query @@ -48,35 +61,58 @@ ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '`val_long`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +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 SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData ORDER BY cate, val -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 2 +1 a 2 +2 a 3 +1 b 1 +2 b 2 +3 b 2 -- !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 schema -struct<> +struct -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 4 +1 a 4 +2 a 2 +1 b 3 +2 b 5 +3 b 3 -- !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 schema -struct<> +struct -- !query output -java.sql.SQLException Error running query: org.apache.spark.sql.AnalysisException: cannot resolve '`val_long`' given input columns: [spark_catalog.default.testdata.key, spark_catalog.default.testdata.value]; line 1 pos 7 +NULL NULL NULL +1 NULL 1 +1 a 4 +1 a 4 +2 a 2147483652 +2147483650 a 2147483650 +NULL b NULL +3 b 2147483653 +2147483650 b 2147483650 -- !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 92da58c27a14..01d1dc26ae29 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 @@ -157,6 +157,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { protected override def sparkConf: SparkConf = super.sparkConf // Fewer shuffle partitions to speed up testing. .set(SQLConf.SHUFFLE_PARTITIONS, 4) + .set(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( 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 f39b2c528895..3fd434a72f0a 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 @@ -21,7 +21,7 @@ import java.io.File import java.sql.Date import org.apache.commons.io.FileUtils -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkException diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index a1b389ecddcc..64d998b89fc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ 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 2ca309408570..208532c3e393 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 @@ -441,17 +441,15 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils wi } test("Hive UDF in group by") { -// withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - withTempView("tab1") { - Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") - sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") - val count = sql("select testUDFToDate(cast(test_date as timestamp))" + - " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate") - assert(count == 1) - } + withTempView("tab1") { + Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") + sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") + val count = sql("select testUDFToDate(cast(test_date as timestamp))" + + " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate") + assert(count == 1) } -// } + } test("SPARK-11522 select input_file_name from non-parquet table") { 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 f21af3185954..642adb6c301e 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 @@ -25,6 +25,7 @@ import java.util.{Locale, Set} import com.google.common.io.Files import org.apache.hadoop.fs.{FileSystem, Path} +import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.sql._ @@ -68,10 +69,20 @@ case class Order( * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is * valid, but Hive currently cannot execute it. */ -abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHiveSingleton { +abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHiveSingleton + with BeforeAndAfter { import hiveContext._ import spark.implicits._ + before { + spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) + } + + after { + spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + SQLConf.get.legacyAllowCastNumericToTimestamp) + } + test("query global temp view") { val df = Seq(1).toDF("i1") df.createGlobalTempView("tbl1") @@ -1171,19 +1182,16 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } test("SPARK-6785: HiveQuerySuite - Date comparison test 2") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - checkAnswer( - sql("SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1"), - Row(false)) - } + checkAnswer( + sql("SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1"), + Row(false)) } test("SPARK-6785: HiveQuerySuite - Date cast") { - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { - // new Date(0) == 1970-01-01 00:00:00.0 GMT == 1969-12-31 16:00:00.0 PST - checkAnswer( - sql( - """ + // new Date(0) == 1970-01-01 00:00:00.0 GMT == 1969-12-31 16:00:00.0 PST + checkAnswer( + sql( + """ | SELECT | CAST(CAST(0 AS timestamp) AS date), | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), @@ -1191,19 +1199,16 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi | CAST(CAST(0 AS timestamp) AS string), | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) | FROM src LIMIT 1 - """. - stripMargin), - Row - ( - Date.valueOf( - "1969-12-31"), + """.stripMargin), + Row( + Date.valueOf("1969-12-31"), String.valueOf("1969-12-31"), Timestamp.valueOf("1969-12-31 16:00:00"), String.valueOf("1969-12-31 16:00:00"), - Timestamp.valueOf("1970-01-01 00:00:00" - ))) - } - } + Timestamp.valueOf("1970-01-01 00:00:00"))) + + } + test("SPARK-8588 HiveTypeCoercion.inConversion fires too early") { val df = createDataFrame(Seq((1, "2014-01-01"), (2, "2015-01-01"), (3, "2016-01-01"))) From e478c90b1fc54745703eabd07e6b1aec62685ec5 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 31 May 2020 14:13:41 +0800 Subject: [PATCH 22/75] fix pyspark test error --- python/pyspark/sql/tests/test_dataframe.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 062e61663a33..1cb3e6c5fb7a 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -604,7 +604,8 @@ def test_to_pandas_from_empty_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_null_dataframe(self): - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False, + "spark.sql.legacy.allowCastNumericToTimestamp":True}): # SPARK-29188 test that toPandas() on a dataframe with only nulls has correct dtypes import numpy as np sql = """ @@ -632,7 +633,8 @@ def test_to_pandas_from_null_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_mixed_dataframe(self): - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False, + "spark.sql.legacy.allowCastNumericToTimestamp":True}): # SPARK-29188 test that toPandas() on a dataframe with some nulls has correct dtypes import numpy as np sql = """ From 734891706b059b763672e516c091f853c3e6674e Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 31 May 2020 14:37:43 +0800 Subject: [PATCH 23/75] fix python style --- python/pyspark/sql/tests/test_dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 1cb3e6c5fb7a..d11fce6be964 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -605,7 +605,7 @@ def test_to_pandas_from_empty_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_null_dataframe(self): with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False, - "spark.sql.legacy.allowCastNumericToTimestamp":True}): + "spark.sql.legacy.allowCastNumericToTimestamp": True}): # SPARK-29188 test that toPandas() on a dataframe with only nulls has correct dtypes import numpy as np sql = """ @@ -634,7 +634,7 @@ def test_to_pandas_from_null_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_mixed_dataframe(self): with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False, - "spark.sql.legacy.allowCastNumericToTimestamp":True}): + "spark.sql.legacy.allowCastNumericToTimestamp": True}): # SPARK-29188 test that toPandas() on a dataframe with some nulls has correct dtypes import numpy as np sql = """ From e50e5cb9461391308d47ac62c0d100451cf351ea Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 1 Jun 2020 11:21:58 +0800 Subject: [PATCH 24/75] annotation examples casting from long to timestamp --- python/pyspark/sql/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 65b902cf3c4d..733a5e5f5e59 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -521,7 +521,7 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving - >>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes') + # >>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 2344e1ffb79beaf5e646dfc8b5b916affe7cef7f Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 1 Jun 2020 22:27:29 +0800 Subject: [PATCH 25/75] fix pyspark test error --- python/pyspark/sql/functions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index de0d38e2aed1..399da7e7c543 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1211,8 +1211,8 @@ def to_date(col, format=None): >>> df.select(to_date(df.t).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] - >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) - >>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect() + # >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + # >>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] """ sc = SparkContext._active_spark_context From 5f138af33d5577234c1811acc11fb18dd97229f5 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 3 Jun 2020 00:32:44 +0800 Subject: [PATCH 26/75] fix to_date using cast long to timestamp --- docs/sql-migration-guide.md | 2 -- python/pyspark/sql/functions.py | 4 ++-- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 4 +++- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index a9ae3c877d11..14416a17e271 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,8 +30,6 @@ license: | - In Spark 3.1, casting numeric to timestamp and will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. - - In Spark 3.1, to_date function with date format as the second parameter will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. - ## Upgrading from Spark SQL 2.4 to 3.0 ### Dataset/DataFrame APIs diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 399da7e7c543..de0d38e2aed1 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1211,8 +1211,8 @@ def to_date(col, format=None): >>> df.select(to_date(df.t).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] - # >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) - # >>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect() + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] """ sc = SparkContext._active_spark_context 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 11a1b05bb66b..a4bc87f10b25 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 @@ -1631,6 +1631,8 @@ case class ToUTCTimestamp(left: Expression, right: Expression) Examples: > SELECT _FUNC_('2009-07-30 04:17:52'); 2009-07-30 + > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd'); + 2016-12-31 """, group = "datetime_funcs", since = "1.5.0") @@ -1640,7 +1642,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr def this(left: Expression, format: Expression) { this(left, Option(format), - Cast(Cast(UnixTimestamp(left, format), TimestampType), DateType)) + Cast(SecondsToTimestamp(UnixTimestamp(left, format)), DateType)) } def this(left: Expression) = { From 1b52fd691a06802a300276d502671024706ea054 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 5 Jun 2020 20:27:21 +0800 Subject: [PATCH 27/75] submit to local --- python/pyspark/sql/functions.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index de0d38e2aed1..415fbe85fce1 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1427,6 +1427,18 @@ def to_utc_timestamp(timestamp, tz): return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz)) +@since(3.1) +def timestamp_seconds(col): + """ + >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) + >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() + [Row(ts=datetime.datetime(2008, 12, 25, 07, 30))] + """ + + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.timestamp_seconds(_to_java_column(col))) + + @since(2.0) @ignore_unicode_prefix def window(timeColumn, windowDuration, slideDuration=None, startTime=None): From 9cf87e8a9ed03255e3f1fc6e12593e6182837f2c Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 01:18:51 +0800 Subject: [PATCH 28/75] extend functions and use new functions --- docs/sql-migration-guide.md | 2 +- python/pyspark/sql/dataframe.py | 2 +- python/pyspark/sql/tests/test_dataframe.py | 8 ++-- .../scala/org/apache/spark/sql/Column.scala | 14 ++++++ .../org/apache/spark/sql/functions.scala | 16 +++++++ .../apache/spark/sql/DateFunctionsSuite.scala | 16 ++----- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 - .../spark/sql/StatisticsCollectionSuite.scala | 16 ++----- .../streaming/MicroBatchExecutionSuite.scala | 11 +---- .../sources/ForeachWriterSuite.scala | 13 ++---- .../sql/expressions/ExpressionInfoSuite.scala | 12 +---- .../streaming/EventTimeWatermarkSuite.scala | 46 ++++++++----------- .../sql/streaming/FileStreamSinkSuite.scala | 5 +- .../FlatMapGroupsWithStateSuite.scala | 20 +++----- .../streaming/StreamingAggregationSuite.scala | 15 ++---- .../StreamingDeduplicationSuite.scala | 23 +++------- .../sql/streaming/StreamingJoinSuite.scala | 39 +++++++--------- .../execution/HiveCompatibilitySuite.scala | 6 +-- .../test/queries/clientpositive/timestamp_3.q | 2 +- .../queries/clientpositive/vectorized_casts.q | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++------ .../sql/hive/execution/HiveUDFSuite.scala | 16 ++----- .../sql/hive/execution/SQLQuerySuite.scala | 23 +++------- 23 files changed, 123 insertions(+), 210 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 14416a17e271..d6e19d1ec47b 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -28,7 +28,7 @@ license: | - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. - - In Spark 3.1, casting numeric to timestamp and will be forbidden by default, user can enable it by setting spark.sql.legacy.allowCastNumericToTimestamp to true, and functions(TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS) are strongly recommended to avoid possible inaccurate scenes, [SPARK-31710](https://issues.apache.org/jira/browse/SPARK-31710) for more details. + - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can setting `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. ## Upgrading from Spark SQL 2.4 to 3.0 diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index cc3a22fff92c..f46f37ade336 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -534,7 +534,7 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving - # >>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes') + >>> sdf.select('name', timestamp_seconds(sdf.time).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index d11fce6be964..30c3fd4c8d16 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -604,8 +604,7 @@ def test_to_pandas_from_empty_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_null_dataframe(self): - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False, - "spark.sql.legacy.allowCastNumericToTimestamp": True}): + 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 = """ @@ -633,8 +632,7 @@ def test_to_pandas_from_null_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_mixed_dataframe(self): - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False, - "spark.sql.legacy.allowCastNumericToTimestamp": True}): + 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 = """ @@ -646,7 +644,7 @@ def test_to_pandas_from_mixed_dataframe(self): CAST(col6 AS DOUBLE) AS double, CAST(col7 AS BOOLEAN) AS boolean, CAST(col8 AS STRING) AS string, - CAST(col9 AS TIMESTAMP) AS timestamp + timestamp_seconds(col9) AS timestamp FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1), (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) """ 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 2144472937f9..0af568061e9b 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 @@ -142,6 +142,8 @@ class TypedColumn[-T, U]( @Stable class Column(val expr: Expression) extends Logging { + import org.apache.spark.sql.functions.withExpr + def this(name: String) = this(name match { case "*" => UnresolvedStar(None) case _ if name.endsWith(".*") => @@ -1045,6 +1047,18 @@ class Column(val expr: Expression) extends Logging { Alias(expr, alias)() } + /** + *usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", + * examples = """ + * Examples: + * > SELECT _FUNC_(1230219000); + * 2008-12-25 07:30:00 + * """, + * group = "datetime_funcs", + * since = "3.1.0") + */ + def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) } + /** * Casts the column to a different data type. * {{{ 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 0cca3e7b47c5..0fbd0075ca63 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 @@ -3358,6 +3358,22 @@ object functions { window(timeColumn, windowDuration, windowDuration, "0 second") } + /** + *usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", + * examples = """ + * Examples: + * > SELECT _FUNC_(1230219000); + * 2008-12-25 07:30:00 + * """, + * group = "datetime_funcs", + * since = "3.1.0") + */ + def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) } + + def array_contains1(column: Column, value: Any): Column = withExpr { + ArrayContains(column.expr, lit(value).expr) + } + ////////////////////////////////////////////////////////////////////////////////////////////// // Collection functions ////////////////////////////////////////////////////////////////////////////////////////////// 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 a99377f83eda..1b617da88a1c 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,8 +23,6 @@ import java.time.{Instant, LocalDateTime, ZoneId} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit -import org.scalatest.BeforeAndAfter - import org.apache.spark.{SparkException, SparkUpgradeException} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CEST, LA} import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -34,17 +32,9 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.DoubleType import org.apache.spark.unsafe.types.CalendarInterval -class DateFunctionsSuite extends QueryTest with SharedSparkSession with BeforeAndAfter{ +class DateFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } test("function current_date") { val df1 = Seq((1, 2), (3, 1)).toDF("a", "b") val d0 = DateTimeUtils.currentDate(ZoneId.systemDefault()) @@ -649,7 +639,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession with BeforeAn val now = sql("select unix_timestamp()").collect().head.getLong(0) checkAnswer( - sql(s"select cast ($now as timestamp)"), + sql(s"select timestamp_seconds($now)"), Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) } } @@ -725,7 +715,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession with BeforeAn 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"))) + df.select(timestamp_seconds(unix_timestamp(col("ss"))))) checkAnswer(df.select(to_timestamp(col("ss"))), Seq( Row(ts1), Row(ts2))) if (legacyParserPolicy == "legacy") { 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 01d1dc26ae29..92da58c27a14 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 @@ -157,7 +157,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { protected override def sparkConf: SparkConf = super.sparkConf // Fewer shuffle partitions to speed up testing. .set(SQLConf.SHUFFLE_PARTITIONS, 4) - .set(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 1f5fc17efdf8..91ec1b5ab293 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -22,7 +22,6 @@ import java.net.URI import java.util.TimeZone import java.util.concurrent.TimeUnit -import org.scalatest.BeforeAndAfter import scala.collection.mutable import org.apache.spark.sql.catalyst.TableIdentifier @@ -30,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} +import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.ArrayData @@ -40,19 +40,9 @@ import org.apache.spark.util.Utils /** * End-to-end suite testing statistics collection and use on both entire table and columns. */ -class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession - with BeforeAndAfter{ +class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession { import testImplicits._ - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } - test("estimates the size of a limit 0 on outer join") { withTempView("test") { Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") @@ -478,7 +468,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared withTable(table) { TimeZone.setDefault(srcTimeZone) spark.range(start, end) - .select('id.cast(TimestampType).cast(t).as(column)) + .select(timestamp_seconds($"id").cast(t).as(column)) .write.saveAsTable(table) sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala index 354592b7fdf2..c0f25e3a5053 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala @@ -19,28 +19,21 @@ package org.apache.spark.sql.execution.streaming import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.StreamTest class MicroBatchExecutionSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - import org.apache.spark.sql.internal.SQLConf - - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) sqlContext.streams.active.foreach(_.stop()) } test("SPARK-24156: do not plan a no-data batch again after it has already been planned") { val inputData = MemoryStream[Int] val df = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index 34f00750dd3c..e87bd11f0dca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -26,22 +26,15 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSparkSession class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeAndAfter { import testImplicits._ - import org.apache.spark.sql.internal.SQLConf - - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) sqlContext.streams.active.foreach(_.stop()) } @@ -170,7 +163,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -204,7 +197,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index a5c5babfdda8..53f975775073 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.expressions -import org.scalatest.BeforeAndAfter import scala.collection.parallel.immutable.ParVector import org.apache.spark.SparkFunSuite @@ -28,16 +27,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils -class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession with BeforeAndAfter { - - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } +class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { test("Replace _FUNC_ in ExpressionInfo") { val info = spark.sessionState.catalog.lookupFunctionInfo(FunctionIdentifier("upper")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index eaae20397822..ae5d066566ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.MemorySink -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ import org.apache.spark.util.Utils @@ -41,13 +41,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche import testImplicits._ - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) sqlContext.streams.active.foreach(_.stop()) } @@ -135,7 +129,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // No event time metrics when there is no watermarking val inputData1 = MemoryStream[Int] val aggWithoutWatermark = inputData1.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) @@ -152,7 +146,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData2 = MemoryStream[Int] val aggWithWatermark = inputData2.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -175,7 +169,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -230,7 +224,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -292,7 +286,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -315,7 +309,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -342,7 +336,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val input = MemoryStream[Long] val aggWithWatermark = input.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "2 years 5 months") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -374,7 +368,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("recovery") { val inputData = MemoryStream[Int] val df = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -409,14 +403,14 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val first = MemoryStream[Int] val firstDf = first.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .select('value) val second = MemoryStream[Int] val secondDf = second.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "5 seconds") .select('value) @@ -486,7 +480,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -511,7 +505,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy($"eventTime") .agg(count("*") as 'count) @@ -550,8 +544,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("the new watermark should override the old one") { val df = MemoryStream[(Long, Long)].toDF() - .withColumn("first", $"_1".cast("timestamp")) - .withColumn("second", $"_2".cast("timestamp")) + .withColumn("first", timestamp_seconds($"_1")) + .withColumn("second", timestamp_seconds($"_2")) .withWatermark("first", "1 minute") .withWatermark("second", "2 minutes") @@ -563,7 +557,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("EventTime watermark should be ignored in batch query.") { val df = testData - .withColumn("eventTime", $"key".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"key")) .withWatermark("eventTime", "1 minute") .select("eventTime") .as[Long] @@ -602,7 +596,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("SPARK-27340 Alias on TimeWindow expression cause watermark metadata lost") { val inputData = MemoryStream[Int] val aliasWindow = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .select(window($"eventTime", "5 seconds") as 'aliasWindow) // Check the eventTime metadata is kept in the top level alias. @@ -632,7 +626,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -768,10 +762,10 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche input1: MemoryStream[Int], input2: MemoryStream[Int]): Dataset[_] = { val df1 = input1.toDF - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") val df2 = input2.toDF - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "15 seconds") df1.union(df2).select($"eventTime".cast("int")) } 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 bdd04b273b6f..a25451bef62f 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 @@ -46,14 +46,11 @@ abstract class FileStreamSinkSuite extends StreamTest { override def beforeAll(): Unit = { super.beforeAll() spark.sessionState.conf.setConf(SQLConf.ORC_IMPLEMENTATION, "native") - spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) } override def afterAll(): Unit = { try { spark.sessionState.conf.unsetConf(SQLConf.ORC_IMPLEMENTATION) - spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) } finally { super.afterAll() } @@ -213,7 +210,7 @@ abstract class FileStreamSinkSuite extends StreamTest { val inputData = MemoryStream[Long] val inputDF = inputData.toDF.toDF("time") val outputDf = inputDF - .selectExpr("CAST(time AS timestamp) AS timestamp") + .selectExpr("timestamp_seconds(time) AS timestamp") .withWatermark("timestamp", "10 seconds") .groupBy(window($"timestamp", "5 seconds")) .count() 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 3fd434a72f0a..e2887e78b050 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 @@ -21,7 +21,7 @@ import java.io.File import java.sql.Date import org.apache.commons.io.FileUtils -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.BeforeAndAfterAll import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkException @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, MemoryStateStore, StateStore, StateStoreId, StateStoreMetrics, UnsafeRowPair} +import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{DataType, IntegerType} @@ -45,22 +46,13 @@ case class RunningCount(count: Long) case class Result(key: Long, count: Int) -class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with BeforeAndAfter { +class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { import testImplicits._ import GroupStateImpl._ import GroupStateTimeout._ import FlatMapGroupsWithStateSuite._ - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } - test("GroupState - get, exists, update, remove") { var state: GroupStateImpl[String] = null @@ -835,7 +827,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with BeforeAndAf val inputData = MemoryStream[(String, Int)] val result = inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime")) .withWatermark("eventTime", "10 seconds") .as[(String, Long)] .groupByKey(_._1) @@ -910,7 +902,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with BeforeAndAf val inputData = MemoryStream[(String, Int)] val result = inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime")) .withWatermark("eventTime", "10 seconds") .as[(String, Long)] .groupByKey(_._1) @@ -1120,7 +1112,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with BeforeAndAf val inputData = MemoryStream[(String, Long)] val result = inputData.toDF().toDF("key", "time") - .selectExpr("key", "cast(time as timestamp) as timestamp") + .selectExpr("key", "timestamp_seconds(time) as timestamp") .withWatermark("timestamp", "10 second") .as[(String, Long)] .groupByKey(x => x._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 9d1e65f777c6..cb69460ca158 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 @@ -23,7 +23,7 @@ import java.util.{Locale, TimeZone} import scala.collection.mutable import org.apache.commons.io.FileUtils -import org.scalatest.{Assertions, BeforeAndAfter} +import org.scalatest.Assertions import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.rdd.BlockRDD @@ -48,19 +48,10 @@ object FailureSingleton { var firstTime = true } -class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions with BeforeAndAfter{ +class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { import testImplicits._ - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } - def executeFuncWithStateVersionSQLConf( stateVersion: Int, confPairs: Seq[(String, String)], @@ -197,7 +188,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions wi testWithAllStateVersions("state metrics - append mode") { val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index 64d998b89fc5..164a00e4377b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ @@ -26,19 +26,10 @@ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAfter{ +class StreamingDeduplicationSuite extends StateStoreMetricsTest { import testImplicits._ - before { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - sqlContext.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } - test("deduplicate with all columns") { val inputData = MemoryStream[String] val result = inputData.toDS().dropDuplicates() @@ -95,7 +86,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAf test("deduplicate with watermark") { val inputData = MemoryStream[Int] val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .select($"eventTime".cast("long").as[Long]) @@ -122,7 +113,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAf test("deduplicate with aggregate - append mode") { val inputData = MemoryStream[Int] val windowedaggregate = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .withWatermark("eventTime", "10 seconds") @@ -238,7 +229,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAf test("SPARK-19841: watermarkPredicate should filter based on keys") { val input = MemoryStream[(Int, Int)] val df = input.toDS.toDF("time", "id") - .withColumn("time", $"time".cast("timestamp")) + .withColumn("time", timestamp_seconds($"time")) .withWatermark("time", "1 second") .dropDuplicates("id", "time") // Change the column positions .select($"id") @@ -257,7 +248,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAf test("SPARK-21546: dropDuplicates should ignore watermark when it's not a key") { val input = MemoryStream[(Int, Int)] val df = input.toDS.toDF("id", "time") - .withColumn("time", $"time".cast("timestamp")) + .withColumn("time", timestamp_seconds($"time")) .withWatermark("time", "1 second") .dropDuplicates("id") .select($"id", $"time".cast("long")) @@ -273,7 +264,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAf def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .select($"eventTime".cast("long").as[Long]) 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 82129dc6b993..e8629df2b656 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 @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreProviderId} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -47,12 +46,9 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with before { SparkSession.setActiveSession(spark) // set this before force initializing 'joinExec' spark.streams.stateStoreCoordinator // initialize the lazy coordinator - spark.conf.set(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key, true) } after { - spark.conf.set(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key, - SQLConf.get.legacyAllowCastNumericToTimestamp) StateStore.stop() } @@ -91,11 +87,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val input2 = MemoryStream[Int] val df1 = input1.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('value as "key", timestamp_seconds($"value") as "timestamp", + ('value * 2) as "leftValue") .select('key, window('timestamp, "10 second"), 'leftValue) val df2 = input2.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", + .select('value as "key", timestamp_seconds($"value") as "timestamp", ('value * 3) as "rightValue") .select('key, window('timestamp, "10 second"), 'rightValue) @@ -131,12 +128,13 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val input2 = MemoryStream[Int] val df1 = input1.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('value as "key", timestamp_seconds($"value") as "timestamp", + ('value * 2) as "leftValue") .withWatermark("timestamp", "10 seconds") .select('key, window('timestamp, "10 second"), 'leftValue) val df2 = input2.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", + .select('value as "key", timestamp_seconds($"value") as "timestamp", ('value * 3) as "rightValue") .select('key, window('timestamp, "10 second"), 'rightValue) @@ -181,11 +179,11 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -239,11 +237,11 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "20 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "30 seconds") val condition = expr( @@ -429,7 +427,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with test("SPARK-26187 restore the stream-stream inner join query from Spark 2.4") { val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) @@ -493,12 +491,9 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with before { SparkSession.setActiveSession(spark) // set this before force initializing 'joinExec' spark.streams.stateStoreCoordinator // initialize the lazy coordinator - spark.conf.set(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key, true) } after { - spark.conf.set(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key, - SQLConf.get.legacyAllowCastNumericToTimestamp) StateStore.stop() } @@ -507,7 +502,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val df = input.toDF .select( 'value as "key", - 'value.cast("timestamp") as s"${prefix}Time", + timestamp_seconds($"value") as s"${prefix}Time", ('value * multiplier) as s"${prefix}Value") .withWatermark(s"${prefix}Time", "10 seconds") @@ -689,11 +684,11 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -784,7 +779,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) @@ -847,7 +842,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) // we're just flipping "left" and "right" from left outer join and apply right outer join @@ -890,7 +885,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with test("SPARK-26187 restore the stream-stream outer join query from Spark 2.4") { val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) 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 fec5fdca0702..db1f6fbd97d9 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 @@ -39,8 +39,6 @@ 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 originallegacyAllowCastNumericToTimestamp = - TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -60,7 +58,6 @@ 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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) RuleExecutor.resetMetrics() } @@ -71,8 +68,7 @@ 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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - originallegacyAllowCastNumericToTimestamp) + // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) } finally { diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q index 0e1a8d552678..34dd90523180 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q @@ -6,7 +6,7 @@ create table timestamp_3 (t timestamp); alter table timestamp_3 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; insert overwrite table timestamp_3 - select cast(cast('1.3041352164485E9' as double) as timestamp) from src tablesample (1 rows); + select timestamp_seconds(cast('1.3041352164485E9' as double)) from src tablesample (1 rows); select cast(t as boolean) from timestamp_3 limit 1; select cast(t as tinyint) from timestamp_3 limit 1; select cast(t as smallint) from timestamp_3 limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q index 3f818b18534a..98156a126bf3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q @@ -43,7 +43,7 @@ select ,cast (cint as float) ,cast (cdouble as float) -- to timestamp - ,cast (ctinyint as timestamp) + ,timestamp_seconds (ctinyint as timestamp) ,cast (csmallint as timestamp) ,cast (cint as timestamp) ,cast (cbigint as timestamp) 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 447618a9587e..7294394d1a7e 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 @@ -50,8 +50,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd import org.apache.spark.sql.hive.test.TestHive.implicits._ private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled - private val originalLegacyAllowCastNumericToTimestamp = - TestHive.conf.legacyAllowCastNumericToTimestamp def spark: SparkSession = sparkSession @@ -60,7 +58,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd TestHive.setCacheTables(true) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) - TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) } override def afterAll(): Unit = { @@ -68,8 +65,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd TestHive.setCacheTables(false) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) - TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - originalLegacyAllowCastNumericToTimestamp) } finally { super.afterAll() } @@ -206,13 +201,13 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), timestamp_seconds(1)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20") + val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), timestamp_seconds(1)) AS COL20") .collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } @@ -557,36 +552,36 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd // Jdk version leads to different query output for double, so not use createQueryTest here test("timestamp cast #1") { - val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(timestamp_seconds(1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(1 == res.getDouble(0)) } createQueryTest("timestamp cast #2", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(timestamp_seconds(1.2) AS DOUBLE) FROM src LIMIT 1") test("timestamp cast #3") { - val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(timestamp_seconds(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } createQueryTest("timestamp cast #4", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(timestamp_seconds(1.2) AS DOUBLE) FROM src LIMIT 1") test("timestamp cast #5") { - val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(timestamp_seconds(-1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(-1 == res.get(0)) } createQueryTest("timestamp cast #6", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(timestamp_seconds(-1.2) AS DOUBLE) FROM src LIMIT 1") test("timestamp cast #7") { - val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(timestamp_seconds(-1200) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } createQueryTest("timestamp cast #8", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(timestamp_seconds(-1.2) AS DOUBLE) FROM src LIMIT 1") createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") 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 208532c3e393..057f2f4ce01b 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 @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.serde2.{AbstractSerDe, SerDeStats} import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.{LongWritable, Writable} -import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.plans.logical.Project @@ -51,20 +50,11 @@ case class ListStringCaseClass(l: Seq[String]) /** * A test suite for Hive custom UDFs. */ -class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfter{ +class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { import spark.udf import spark.implicits._ - before { - spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } - test("spark sql udf test that returns a struct") { udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) assert(sql( @@ -444,8 +434,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils wi withTempView("tab1") { Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") - val count = sql("select testUDFToDate(cast(test_date as timestamp))" + - " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() + val count = sql("select testUDFToDate(timestamp_seconds(test_date))" + + " from tab1 group by testUDFToDate(timestamp_seconds(test_date))").count() sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate") assert(count == 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 642adb6c301e..8e8e037f6233 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 @@ -25,7 +25,6 @@ import java.util.{Locale, Set} import com.google.common.io.Files import org.apache.hadoop.fs.{FileSystem, Path} -import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.sql._ @@ -69,20 +68,10 @@ case class Order( * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is * valid, but Hive currently cannot execute it. */ -abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHiveSingleton - with BeforeAndAfter { +abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ import spark.implicits._ - before { - spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) - } - - after { - spark.sessionState.conf.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - SQLConf.get.legacyAllowCastNumericToTimestamp) - } - test("query global temp view") { val df = Seq(1).toDF("i1") df.createGlobalTempView("tbl1") @@ -1183,7 +1172,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi test("SPARK-6785: HiveQuerySuite - Date comparison test 2") { checkAnswer( - sql("SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1"), + sql("SELECT CAST(timestamp_seconds(0) AS date) > timestamp_seconds(0) FROM src LIMIT 1"), Row(false)) } @@ -1193,10 +1182,10 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi sql( """ | SELECT - | CAST(CAST(0 AS timestamp) AS date), - | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), - | CAST(0 AS timestamp), - | CAST(CAST(0 AS timestamp) AS string), + | CAST(timestamp_seconds(0) AS date), + | CAST(CAST(timestamp_seconds(0) AS date) AS string), + | timestamp_seconds(0), + | CAST(timestamp_seconds(0) AS string), | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) | FROM src LIMIT 1 """.stripMargin), From cb21d5de9c334b2fb908d480e53988b00be755f2 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 01:37:09 +0800 Subject: [PATCH 29/75] fix code style.etc --- .../scala/org/apache/spark/sql/Column.scala | 14 ---------- .../org/apache/spark/sql/functions.scala | 8 +++--- .../queries/clientpositive/vectorized_casts.q | 28 +++++++++---------- 3 files changed, 18 insertions(+), 32 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 0af568061e9b..2144472937f9 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 @@ -142,8 +142,6 @@ class TypedColumn[-T, U]( @Stable class Column(val expr: Expression) extends Logging { - import org.apache.spark.sql.functions.withExpr - def this(name: String) = this(name match { case "*" => UnresolvedStar(None) case _ if name.endsWith(".*") => @@ -1047,18 +1045,6 @@ class Column(val expr: Expression) extends Logging { Alias(expr, alias)() } - /** - *usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", - * examples = """ - * Examples: - * > SELECT _FUNC_(1230219000); - * 2008-12-25 07:30:00 - * """, - * group = "datetime_funcs", - * since = "3.1.0") - */ - def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) } - /** * Casts the column to a different data type. * {{{ 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 0fbd0075ca63..0a1f67aa9fe5 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 @@ -3358,15 +3358,15 @@ object functions { window(timeColumn, windowDuration, windowDuration, "0 second") } - /** - *usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", + /** + * usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", * examples = """ * Examples: * > SELECT _FUNC_(1230219000); * 2008-12-25 07:30:00 * """, - * group = "datetime_funcs", - * since = "3.1.0") + * @group = "datetime_funcs", + * @since = "3.1.0") */ def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) } diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q index 98156a126bf3..0f0778e830a3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q @@ -43,14 +43,14 @@ select ,cast (cint as float) ,cast (cdouble as float) -- to timestamp - ,timestamp_seconds (ctinyint as timestamp) - ,cast (csmallint as timestamp) - ,cast (cint as timestamp) - ,cast (cbigint as timestamp) - ,cast (cfloat as timestamp) - ,cast (cdouble as timestamp) + ,timestamp_seconds (ctinyint) + ,timestamp_seconds (csmallint) + ,catimestamp_secondsst (cint) + ,timestamp_seconds (cbigint) + ,timestamp_seconds (cfloat) + ,timestamp_seconds (cdouble) ,cast (cboolean1 as timestamp) - ,cast (cbigint * 0 as timestamp) + ,timestamp_seconds (cbigint * 0) ,cast (ctimestamp1 as timestamp) ,cast (cstring1 as timestamp) ,cast (substr(cstring1, 1, 1) as timestamp) @@ -115,14 +115,14 @@ select ,cast (cint as float) ,cast (cdouble as float) -- to timestamp - ,cast (ctinyint as timestamp) - ,cast (csmallint as timestamp) - ,cast (cint as timestamp) - ,cast (cbigint as timestamp) - ,cast (cfloat as timestamp) - ,cast (cdouble as timestamp) + ,timestamp_seconds (ctinyint) + ,timestamp_seconds (csmallint) + ,timestamp_seconds (cintp) + ,timestamp_seconds (cbigint) + ,timestamp_seconds (cfloat) + ,timestamp_seconds (cdouble) ,cast (cboolean1 as timestamp) - ,cast (cbigint * 0 as timestamp) + ,timestamp_seconds (cbigint * 0 as timestamp) ,cast (ctimestamp1 as timestamp) ,cast (cstring1 as timestamp) ,cast (substr(cstring1, 1, 1) as timestamp) From 726371cf531b042ba61b100c5f31fab4907dfd0f Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 01:58:01 +0800 Subject: [PATCH 30/75] fix code style --- .../apache/spark/sql/streaming/StreamingJoinSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 e8629df2b656..fe5f34917113 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 @@ -183,7 +183,8 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, timestamp_seconds($"time") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -241,7 +242,8 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with .withWatermark("leftTime", "20 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, timestamp_seconds($"time") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "30 seconds") val condition = expr( @@ -688,7 +690,8 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, timestamp_seconds($"time") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = From 8ed4fc41a43442864cdb0fa2e0160fa5e80f374e Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 08:13:32 +0800 Subject: [PATCH 31/75] fix doc format.etc --- docs/sql-migration-guide.md | 2 +- .../src/main/scala/org/apache/spark/sql/functions.scala | 7 +------ .../ql/src/test/queries/clientpositive/vectorized_casts.q | 6 +++--- 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d6e19d1ec47b..02e000bac64f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -27,7 +27,7 @@ license: | - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. - + - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can setting `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. ## Upgrading from Spark SQL 2.4 to 3.0 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 0a1f67aa9fe5..0a3af5111cd1 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 @@ -3359,12 +3359,7 @@ object functions { } /** - * usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", - * examples = """ - * Examples: - * > SELECT _FUNC_(1230219000); - * 2008-12-25 07:30:00 - * """, + * Creates timestamp from the number of seconds since UTC epoch.", * @group = "datetime_funcs", * @since = "3.1.0") */ diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q index 0f0778e830a3..a69fd084f46b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q @@ -45,7 +45,7 @@ select -- to timestamp ,timestamp_seconds (ctinyint) ,timestamp_seconds (csmallint) - ,catimestamp_secondsst (cint) + ,timestamp_seconds (cint) ,timestamp_seconds (cbigint) ,timestamp_seconds (cfloat) ,timestamp_seconds (cdouble) @@ -117,12 +117,12 @@ select -- to timestamp ,timestamp_seconds (ctinyint) ,timestamp_seconds (csmallint) - ,timestamp_seconds (cintp) + ,timestamp_seconds (cint) ,timestamp_seconds (cbigint) ,timestamp_seconds (cfloat) ,timestamp_seconds (cdouble) ,cast (cboolean1 as timestamp) - ,timestamp_seconds (cbigint * 0 as timestamp) + ,timestamp_seconds (cbigint * 0) ,cast (ctimestamp1 as timestamp) ,cast (cstring1 as timestamp) ,cast (substr(cstring1, 1, 1) as timestamp) From 96c197cdac39d6b91f81ef6bee82f4a03dcb3979 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 09:03:10 +0800 Subject: [PATCH 32/75] fix testcase error --- .../sql-tests/inputs/udf/udf-window.sql | 16 ++++++++-------- .../test/resources/sql-tests/inputs/window.sql | 16 ++++++++-------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql index bcbf87f8a04c..1659f1c81959 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql @@ -1,15 +1,15 @@ --This test file was converted from window.sql. -- Test data. 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"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween 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 3d05dfda6c3f..72d812d6a4e4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -5,15 +5,15 @@ -- Test data. 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"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween From 3d1819d3e1b19d750906580b8b1ba98a1501faa5 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 10:42:41 +0800 Subject: [PATCH 33/75] failed test case --- .../sql-tests/results/udf/udf-window.sql.out | 16 ++++++++-------- .../resources/sql-tests/results/window.sql.out | 16 ++++++++-------- 2 files changed, 16 insertions(+), 16 deletions(-) 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 a915c1bd6c71..a84070535b65 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 @@ -4,15 +4,15 @@ -- !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"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> 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 625088f90ced..ede044a44fda 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 @@ -4,15 +4,15 @@ -- !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"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> From be88f0157fa3cb48f1b9d69998ffffea13fc5311 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 6 Jun 2020 16:00:28 +0800 Subject: [PATCH 34/75] fix testcase error --- .../expressions/datetimeExpressions.scala | 35 +++++++++++++++++-- .../expressions/DateExpressionsSuite.scala | 16 +++++++++ .../test/queries/clientpositive/decimal_1.q | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 4 files changed, 50 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 def571329e97..a228ff081e03 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 @@ -411,19 +411,48 @@ abstract class NumberToTimestampBase extends UnaryExpression protected def upScaleFactor: Long - override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) + override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) override def dataType: DataType = TimestampType override def nullSafeEval(input: Any): Any = { - Math.multiplyExact(input.asInstanceOf[Number].longValue(), upScaleFactor) + child.dataType match { + case ByteType | ShortType | IntegerType | LongType => + Math.multiplyExact(input.asInstanceOf[Number].longValue(), upScaleFactor).longValue() + case DecimalType() => + (input.asInstanceOf[Decimal].toBigDecimal * upScaleFactor).longValue() + case FloatType => + if (input.asInstanceOf[Float].isNaN || input.asInstanceOf[Float].isInfinite) null + (input.asInstanceOf[Float] * upScaleFactor).longValue() + case DoubleType => + if (input.asInstanceOf[Double].isNaN || input.asInstanceOf[Double].isInfinite) null + (input.asInstanceOf[Double] * upScaleFactor).longValue() + } } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { if (upScaleFactor == 1) { defineCodeGen(ctx, ev, c => c) } else { - defineCodeGen(ctx, ev, c => s"java.lang.Math.multiplyExact($c, ${upScaleFactor}L)") + child.dataType match { + case ByteType | ShortType | IntegerType | LongType => + defineCodeGen(ctx, ev, c => + s"java.lang.Math.multiplyExact($c, ${upScaleFactor}L)") + case DecimalType() => + val block = inline"new java.math.BigDecimal($upScaleFactor)" + defineCodeGen(ctx, ev, c => + s"($c.toBigDecimal().bigDecimal().multiply($block)).longValue()") + case FloatType => + val caf = child.eval().asInstanceOf[Float] + if (caf.isNaN || caf.isInfinite) ExprCode.forNullValue(LongType) + else defineCodeGen(ctx, ev, c => + s"(long)($c * ${upScaleFactor})") + case DoubleType => + val cad = child.eval().asInstanceOf[Double] + if (cad.isNaN || cad.isInfinite) ExprCode.forNullValue(LongType) + else defineCodeGen(ctx, ev, c => + s"(long)($c * ${upScaleFactor})") + } } } } 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 2dc5990eb610..7ab039cbc311 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 @@ -1149,6 +1149,22 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-31710:Adds TIMESTAMP_SECONDS, " + "TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions") { + checkEvaluation(SecondsToTimestamp(Literal(null, DoubleType)), null) + checkEvaluation(SecondsToTimestamp(Literal(null, FloatType)), null) + checkEvaluation(SecondsToTimestamp(Literal(1.3041352164485E9)), + (1.3041352164485E9 * MICROS_PER_SECOND).longValue()) + checkEvaluation(SecondsToTimestamp(Literal(2.3f)), (2.3d * MICROS_PER_SECOND).longValue()) + checkEvaluation(SecondsToTimestamp(Literal(2.3f)), (2.3f * MICROS_PER_SECOND).longValue()) + checkEvaluation(SecondsToTimestamp(Literal(Decimal(BigDecimal(23.53333333333333333), 8, 6))), + (BigDecimal(23.53333333333333333) * MICROS_PER_SECOND).longValue()) + + checkEvaluation(MillisToTimestamp(Literal(null, DoubleType)), null) + checkEvaluation(MillisToTimestamp(Literal(null, FloatType)), null) + checkEvaluation(MillisToTimestamp(Literal(2.3f)), (2.3d * MICROS_PER_MILLIS).longValue()) + checkEvaluation(MillisToTimestamp(Literal(2.3f)), (2.3f * MICROS_PER_MILLIS).longValue()) + checkEvaluation(MillisToTimestamp(Literal(Decimal(BigDecimal(23.53333333333333333), 8, 6))), + (BigDecimal(23.53333333333333333) * MICROS_PER_MILLIS).longValue()) + checkEvaluation(SecondsToTimestamp(Literal(1230219000)), 1230219000L * MICROS_PER_SECOND) checkEvaluation(SecondsToTimestamp(Literal(-1230219000)), -1230219000L * MICROS_PER_SECOND) checkEvaluation(SecondsToTimestamp(Literal(null, IntegerType)), null) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q index f52b1923eb06..077efca825e1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q @@ -17,6 +17,6 @@ select cast(t as bigint) from decimal_1; select cast(t as float) from decimal_1; select cast(t as double) from decimal_1; select cast(t as string) from decimal_1; -select cast(t as timestamp) from decimal_1; +select timestamp_seconds(t) from decimal_1; drop table decimal_1; 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 7294394d1a7e..c6ada67f095a 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 @@ -201,7 +201,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), timestamp_seconds(1)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), SecondsToTimestamp(1)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) From fd7815aebefbca686c74349025afc817f4979d6c Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 12:52:03 +0800 Subject: [PATCH 35/75] fix find no test result problem --- .../constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 | 1 - .../timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 | 1 - .../timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 | 1 - .../timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 | 1 - .../timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 | 1 - .../apache/spark/sql/hive/execution/HiveComparisonTest.scala | 1 + .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 3 +-- 7 files changed, 2 insertions(+), 7 deletions(-) delete mode 100644 sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 b/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 deleted file mode 100644 index a01c2622c68e..000000000000 --- a/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 +++ /dev/null @@ -1 +0,0 @@ -1 NULL 1 NULL 1.0 NULL true NULL 1 NULL 1.0 NULL 1 NULL 1 NULL 1 NULL 1970-01-01 NULL NULL 1 NULL diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da887..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da887..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 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 2e4c01830432..1a6670e0e501 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 @@ -320,6 +320,7 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => logDebug(s"Looking for cached answer file $cachedAnswerFile.") + logError(s"sss" + cachedAnswerFile) if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else { 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 c6ada67f095a..3a1c39eea347 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 @@ -201,7 +201,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), SecondsToTimestamp(1)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), timestamp_seconds(1)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) @@ -555,7 +555,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd val res = sql("SELECT CAST(timestamp_seconds(1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(1 == res.getDouble(0)) } - createQueryTest("timestamp cast #2", "SELECT CAST(timestamp_seconds(1.2) AS DOUBLE) FROM src LIMIT 1") From e14c4f92ef7ea253bf524af2a5646528ef8acb43 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 13:20:15 +0800 Subject: [PATCH 36/75] fix find no test result problem --- .../golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 | 1 - .../org/apache/spark/sql/hive/execution/HiveComparisonTest.scala | 1 - .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 1 - 3 files changed, 3 deletions(-) delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 deleted file mode 100644 index c4a17c1b14c8..000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 +++ /dev/null @@ -1 +0,0 @@ -1969-12-31 16:00:17.29 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 1a6670e0e501..2e4c01830432 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 @@ -320,7 +320,6 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => logDebug(s"Looking for cached answer file $cachedAnswerFile.") - logError(s"sss" + cachedAnswerFile) if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else { 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 3a1c39eea347..a5ba0f90b29b 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 @@ -562,7 +562,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd val res = sql("SELECT CAST(timestamp_seconds(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } - createQueryTest("timestamp cast #4", "SELECT CAST(timestamp_seconds(1.2) AS DOUBLE) FROM src LIMIT 1") From 7cb0a548acbb29948a0f0299bb1b9ba073e9f73d Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 13:55:22 +0800 Subject: [PATCH 37/75] fix testcase error --- .gitignore | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 198fdee39be9..bf8c1b958eff 100644 --- a/.gitignore +++ b/.gitignore @@ -87,6 +87,7 @@ metastore_db/ sql/hive-thriftserver/test_warehouses warehouse/ spark-warehouse/ +!sql/hive/src/test/resources/golden/* # For R session data .RData @@ -98,4 +99,4 @@ spark-warehouse/ .Rproj.user # For SBT -.jvmopts +.jvmopts \ No newline at end of file From 7ad82da701b10f17af2a1ba764fc8afc2a11ff7b Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 14:04:13 +0800 Subject: [PATCH 38/75] fix found no test result file --- .../constant null testing-0-4af7ceacb7569cf8d411b33276a5804b | 1 + .../golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 | 1 + .../golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 | 1 + .../golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 | 1 + .../golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 | 1 + .../golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 | 1 + 6 files changed, 6 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/constant null testing-0-4af7ceacb7569cf8d411b33276a5804b create mode 100644 sql/hive/src/test/resources/golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-4af7ceacb7569cf8d411b33276a5804b b/sql/hive/src/test/resources/golden/constant null testing-0-4af7ceacb7569cf8d411b33276a5804b new file mode 100644 index 000000000000..a01c2622c68e --- /dev/null +++ b/sql/hive/src/test/resources/golden/constant null testing-0-4af7ceacb7569cf8d411b33276a5804b @@ -0,0 +1 @@ +1 NULL 1 NULL 1.0 NULL true NULL 1 NULL 1.0 NULL 1 NULL 1 NULL 1 NULL 1970-01-01 NULL NULL 1 NULL diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 b/sql/hive/src/test/resources/golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 new file mode 100644 index 000000000000..c4a17c1b14c8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 @@ -0,0 +1 @@ +1969-12-31 16:00:17.29 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 b/sql/hive/src/test/resources/golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 new file mode 100644 index 000000000000..5625e59da887 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 new file mode 100644 index 000000000000..5625e59da887 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 b/sql/hive/src/test/resources/golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 new file mode 100644 index 000000000000..1d94c8a014fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 @@ -0,0 +1 @@ +-1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 new file mode 100644 index 000000000000..1d94c8a014fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 @@ -0,0 +1 @@ +-1.2 From 00ed960accee0be4ab92208d85118098f851b782 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 22:46:56 +0800 Subject: [PATCH 39/75] fix test case error --- .../test/resources/sql-tests/results/ansi/datetime.sql.out | 6 +++--- .../resources/sql-tests/results/datetime-legacy.sql.out | 6 +++--- .../src/test/resources/sql-tests/results/datetime.sql.out | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index a4e6e79b4573..a8d98b1c12e8 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -5,7 +5,7 @@ -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL @@ -13,7 +13,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL @@ -21,7 +21,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 38d078838ebe..b3dd326c12a4 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -5,7 +5,7 @@ -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL @@ -13,7 +13,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL @@ -21,7 +21,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 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 dc4220ff6226..273481f8fd75 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 @@ -5,7 +5,7 @@ -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL @@ -13,7 +13,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL @@ -21,7 +21,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL From b8b29194aad6155e67e0984b228d961d71c43afc Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 23:09:12 +0800 Subject: [PATCH 40/75] fix test case fail --- streaming/src/test/resources/spark-version-info.properties | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 streaming/src/test/resources/spark-version-info.properties diff --git a/streaming/src/test/resources/spark-version-info.properties b/streaming/src/test/resources/spark-version-info.properties new file mode 100644 index 000000000000..69db78bdf332 --- /dev/null +++ b/streaming/src/test/resources/spark-version-info.properties @@ -0,0 +1,6 @@ +version=3.1.0 +user=dcadmin +revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f +branch=31710-fix-compatibility +date=2020-06-03T13:50:55Z +url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From eeb0a61498556056aed9f94a7e9c864bd23e6ce6 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 7 Jun 2020 23:10:07 +0800 Subject: [PATCH 41/75] fix test case fail --- streaming/src/test/resources/spark-version-info.properties | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 streaming/src/test/resources/spark-version-info.properties diff --git a/streaming/src/test/resources/spark-version-info.properties b/streaming/src/test/resources/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/streaming/src/test/resources/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From 409c821281b84a970c10878974c3ddd39f34d381 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 8 Jun 2020 09:01:39 +0800 Subject: [PATCH 42/75] fix check spark streaming ui timeout --- .../scala/org/apache/spark/streaming/UISeleniumSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 952ef6c374f3..7ba3ac28394b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -104,7 +104,7 @@ class UISeleniumSuite find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) } - eventually(timeout(10.seconds), interval(500.milliseconds)) { + eventually(timeout(30.seconds), interval(500.milliseconds)) { // check whether streaming page exists go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq @@ -128,7 +128,6 @@ class UISeleniumSuite h4Text.exists(_.matches("Running Batches \\(\\d+\\)")) should be (true) h4Text.exists(_.matches("Waiting Batches \\(\\d+\\)")) should be (true) h4Text.exists(_.matches("Completed Batches \\(last \\d+ out of \\d+\\)")) should be (true) - val arrow = 0x25BE.toChar findAll(cssSelector("""#runningBatches-table th""")).map(_.text).toList should be { List(s"Batch Time $arrow", "Records", "Scheduling Delay", "Processing Time", From ec2cf54b6d566dd7afcd65753374c1dd5dc8d47f Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 8 Jun 2020 09:02:16 +0800 Subject: [PATCH 43/75] fix additional space --- .../test/scala/org/apache/spark/streaming/UISeleniumSuite.scala | 1 + 1 file changed, 1 insertion(+) 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 7ba3ac28394b..64d79bbb2d2c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -128,6 +128,7 @@ class UISeleniumSuite h4Text.exists(_.matches("Running Batches \\(\\d+\\)")) should be (true) h4Text.exists(_.matches("Waiting Batches \\(\\d+\\)")) should be (true) h4Text.exists(_.matches("Completed Batches \\(last \\d+ out of \\d+\\)")) should be (true) + val arrow = 0x25BE.toChar findAll(cssSelector("""#runningBatches-table th""")).map(_.text).toList should be { List(s"Batch Time $arrow", "Records", "Scheduling Delay", "Processing Time", From 3fd6d02d66aa2abfbe80450366a1d25a332e66ee Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 8 Jun 2020 12:16:19 +0800 Subject: [PATCH 44/75] fix test case fail --- python/pyspark/sql/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f46f37ade336..87349c407c95 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -534,7 +534,7 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving - >>> sdf.select('name', timestamp_seconds(sdf.time).withWatermark('time', '10 minutes') + >>> sdf.select('name', timestamp_seconds(sdf.time)).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 47f210cdea6aacc193474bd4c7cf901e881f0c16 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 8 Jun 2020 18:07:58 +0800 Subject: [PATCH 45/75] fix test error.etc --- python/pyspark/sql/dataframe.py | 2 +- python/pyspark/sql/functions.py | 1 + .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 87349c407c95..abdf4107597e 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -533,7 +533,7 @@ def withWatermark(self, eventTime, delayThreshold): (e.g. "1 minute" or "5 hours"). .. note:: Evolving - + >>> from pyspark.sql.functions import timestamp_seconds >>> sdf.select('name', timestamp_seconds(sdf.time)).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 415fbe85fce1..7ca2ed9cde40 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1430,6 +1430,7 @@ def to_utc_timestamp(timestamp, tz): @since(3.1) def timestamp_seconds(col): """ + >>> from pyspark.sql.functions import timestamp_seconds >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() [Row(ts=datetime.datetime(2008, 12, 25, 07, 30))] 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 11d8c413070a..5576e71b5702 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 @@ -271,7 +271,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + "you can enable the casting by setting " + s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + - "but we strongly recommand using function " + + "but we strongly recommend using function " + "TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead." } else { s"cannot cast ${child.dataType.catalogString} to ${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 d04eed373fc4..b315b996085f 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 @@ -2598,8 +2598,8 @@ object SQLConf { val LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP = buildConf("spark.sql.legacy.allowCastNumericToTimestamp") .internal() - .doc("When true, allow cast numeric to timestamp, but for integral numbers," + - "Hive treats it as milliseconds, Spark SQL treats n as seconds") + .doc("When true, allow casting numeric to timestamp," + + "when false, forbid the cast, more details in SPARK-31710") .version("3.1.0") .booleanConf .createWithDefault(false) From 860500d74162ea0212c08701dfee23589314eee2 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 8 Jun 2020 18:34:45 +0800 Subject: [PATCH 46/75] fix test error.etc --- .gitignore | 3 +-- docs/sql-migration-guide.md | 2 +- python/pyspark/sql/dataframe.py | 1 + 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitignore b/.gitignore index bf8c1b958eff..198fdee39be9 100644 --- a/.gitignore +++ b/.gitignore @@ -87,7 +87,6 @@ metastore_db/ sql/hive-thriftserver/test_warehouses warehouse/ spark-warehouse/ -!sql/hive/src/test/resources/golden/* # For R session data .RData @@ -99,4 +98,4 @@ spark-warehouse/ .Rproj.user # For SBT -.jvmopts \ No newline at end of file +.jvmopts diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 02e000bac64f..df4c75bc89ef 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -28,7 +28,7 @@ license: | - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. - - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can setting `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. + - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can set `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. ## Upgrading from Spark SQL 2.4 to 3.0 diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index abdf4107597e..dc4812cffae9 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -533,6 +533,7 @@ def withWatermark(self, eventTime, delayThreshold): (e.g. "1 minute" or "5 hours"). .. note:: Evolving + >>> from pyspark.sql.functions import timestamp_seconds >>> sdf.select('name', timestamp_seconds(sdf.time)).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] From 4c7ddb41e7e9d4a601a2cb2a508d5e81d399e9fc Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 9 Jun 2020 11:20:12 +0800 Subject: [PATCH 47/75] revert support fraction types.etc --- python/pyspark/sql/dataframe.py | 2 +- .../expressions/datetimeExpressions.scala | 35 ++----------------- .../expressions/DateExpressionsSuite.scala | 16 --------- .../sql-tests/results/ansi/datetime.sql.out | 6 ++-- .../sql-tests/results/datetime-legacy.sql.out | 6 ++-- .../sql-tests/results/datetime.sql.out | 6 ++-- .../execution/HiveCompatibilitySuite.scala | 6 +++- ...esting-0-237a6af90a857da1efcbe98f6bbbf9d6} | 0 ...mal_1-14-e45935cfffb9045394e804d0d1fc52f0} | 0 ...ast #2-0-732ed232ac592c5e7f7c913a88874fd2} | 0 ...ast #4-0-732ed232ac592c5e7f7c913a88874fd2} | 0 ...ast #6-0-6d2da5cfada03605834e38bc4075bc79} | 0 ...ast #8-0-6d2da5cfada03605834e38bc4075bc79} | 0 .../test/queries/clientpositive/decimal_1.q | 2 +- .../test/queries/clientpositive/timestamp_3.q | 2 +- .../queries/clientpositive/vectorized_casts.q | 28 +++++++-------- .../sql/hive/execution/HiveQuerySuite.scala | 27 ++++++++------ 17 files changed, 51 insertions(+), 85 deletions(-) rename sql/hive/src/test/resources/golden/{constant null testing-0-4af7ceacb7569cf8d411b33276a5804b => constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 => decimal_1-14-e45935cfffb9045394e804d0d1fc52f0} (100%) rename sql/hive/src/test/resources/golden/{timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 => timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2} (100%) rename sql/hive/src/test/resources/golden/{timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 => timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2} (100%) rename sql/hive/src/test/resources/golden/{timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 => timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79} (100%) rename sql/hive/src/test/resources/golden/{timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 => timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79} (100%) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index dc4812cffae9..237e8ad794fa 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -535,7 +535,7 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving >>> from pyspark.sql.functions import timestamp_seconds - >>> sdf.select('name', timestamp_seconds(sdf.time)).withWatermark('time', '10 minutes') + >>> sdf.select('name', timestamp_seconds($"time")).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: 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 a228ff081e03..def571329e97 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 @@ -411,48 +411,19 @@ abstract class NumberToTimestampBase extends UnaryExpression protected def upScaleFactor: Long - override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) + override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) override def dataType: DataType = TimestampType override def nullSafeEval(input: Any): Any = { - child.dataType match { - case ByteType | ShortType | IntegerType | LongType => - Math.multiplyExact(input.asInstanceOf[Number].longValue(), upScaleFactor).longValue() - case DecimalType() => - (input.asInstanceOf[Decimal].toBigDecimal * upScaleFactor).longValue() - case FloatType => - if (input.asInstanceOf[Float].isNaN || input.asInstanceOf[Float].isInfinite) null - (input.asInstanceOf[Float] * upScaleFactor).longValue() - case DoubleType => - if (input.asInstanceOf[Double].isNaN || input.asInstanceOf[Double].isInfinite) null - (input.asInstanceOf[Double] * upScaleFactor).longValue() - } + Math.multiplyExact(input.asInstanceOf[Number].longValue(), upScaleFactor) } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { if (upScaleFactor == 1) { defineCodeGen(ctx, ev, c => c) } else { - child.dataType match { - case ByteType | ShortType | IntegerType | LongType => - defineCodeGen(ctx, ev, c => - s"java.lang.Math.multiplyExact($c, ${upScaleFactor}L)") - case DecimalType() => - val block = inline"new java.math.BigDecimal($upScaleFactor)" - defineCodeGen(ctx, ev, c => - s"($c.toBigDecimal().bigDecimal().multiply($block)).longValue()") - case FloatType => - val caf = child.eval().asInstanceOf[Float] - if (caf.isNaN || caf.isInfinite) ExprCode.forNullValue(LongType) - else defineCodeGen(ctx, ev, c => - s"(long)($c * ${upScaleFactor})") - case DoubleType => - val cad = child.eval().asInstanceOf[Double] - if (cad.isNaN || cad.isInfinite) ExprCode.forNullValue(LongType) - else defineCodeGen(ctx, ev, c => - s"(long)($c * ${upScaleFactor})") - } + defineCodeGen(ctx, ev, c => s"java.lang.Math.multiplyExact($c, ${upScaleFactor}L)") } } } 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 7ab039cbc311..2dc5990eb610 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 @@ -1149,22 +1149,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-31710:Adds TIMESTAMP_SECONDS, " + "TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions") { - checkEvaluation(SecondsToTimestamp(Literal(null, DoubleType)), null) - checkEvaluation(SecondsToTimestamp(Literal(null, FloatType)), null) - checkEvaluation(SecondsToTimestamp(Literal(1.3041352164485E9)), - (1.3041352164485E9 * MICROS_PER_SECOND).longValue()) - checkEvaluation(SecondsToTimestamp(Literal(2.3f)), (2.3d * MICROS_PER_SECOND).longValue()) - checkEvaluation(SecondsToTimestamp(Literal(2.3f)), (2.3f * MICROS_PER_SECOND).longValue()) - checkEvaluation(SecondsToTimestamp(Literal(Decimal(BigDecimal(23.53333333333333333), 8, 6))), - (BigDecimal(23.53333333333333333) * MICROS_PER_SECOND).longValue()) - - checkEvaluation(MillisToTimestamp(Literal(null, DoubleType)), null) - checkEvaluation(MillisToTimestamp(Literal(null, FloatType)), null) - checkEvaluation(MillisToTimestamp(Literal(2.3f)), (2.3d * MICROS_PER_MILLIS).longValue()) - checkEvaluation(MillisToTimestamp(Literal(2.3f)), (2.3f * MICROS_PER_MILLIS).longValue()) - checkEvaluation(MillisToTimestamp(Literal(Decimal(BigDecimal(23.53333333333333333), 8, 6))), - (BigDecimal(23.53333333333333333) * MICROS_PER_MILLIS).longValue()) - checkEvaluation(SecondsToTimestamp(Literal(1230219000)), 1230219000L * MICROS_PER_SECOND) checkEvaluation(SecondsToTimestamp(Literal(-1230219000)), -1230219000L * MICROS_PER_SECOND) checkEvaluation(SecondsToTimestamp(Literal(null, IntegerType)), null) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index a8d98b1c12e8..a4e6e79b4573 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -5,7 +5,7 @@ -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL @@ -13,7 +13,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL @@ -21,7 +21,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index b3dd326c12a4..38d078838ebe 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -5,7 +5,7 @@ -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL @@ -13,7 +13,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL @@ -21,7 +21,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 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 273481f8fd75..dc4220ff6226 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 @@ -5,7 +5,7 @@ -- !query select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) -- !query schema -struct +struct -- !query output 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL @@ -13,7 +13,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL @@ -21,7 +21,7 @@ struct +struct -- !query output 2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL 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 db1f6fbd97d9..d6ba756f29f2 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 @@ -39,6 +39,8 @@ 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 originalLegacyAllowCastNumericToTimestamp = + TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -58,6 +60,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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) RuleExecutor.resetMetrics() } @@ -68,7 +71,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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + originalLegacyAllowCastNumericToTimestamp) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) } finally { diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-4af7ceacb7569cf8d411b33276a5804b b/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 similarity index 100% rename from sql/hive/src/test/resources/golden/constant null testing-0-4af7ceacb7569cf8d411b33276a5804b rename to sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-14-56d99c1ab4ade4d1ab585bc87c5dc123 rename to sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 b/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #2-0-855cf6bdc7ccbab85d03bb4c5cdf0525 rename to sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #4-0-855cf6bdc7ccbab85d03bb4c5cdf0525 rename to sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 b/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #6-0-211be28810b6ccbe978cca47319526a3 rename to sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #8-0-211be28810b6ccbe978cca47319526a3 rename to sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q index 077efca825e1..f52b1923eb06 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q @@ -17,6 +17,6 @@ select cast(t as bigint) from decimal_1; select cast(t as float) from decimal_1; select cast(t as double) from decimal_1; select cast(t as string) from decimal_1; -select timestamp_seconds(t) from decimal_1; +select cast(t as timestamp) from decimal_1; drop table decimal_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q index 34dd90523180..0e1a8d552678 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q @@ -6,7 +6,7 @@ create table timestamp_3 (t timestamp); alter table timestamp_3 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; insert overwrite table timestamp_3 - select timestamp_seconds(cast('1.3041352164485E9' as double)) from src tablesample (1 rows); + select cast(cast('1.3041352164485E9' as double) as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_3 limit 1; select cast(t as tinyint) from timestamp_3 limit 1; select cast(t as smallint) from timestamp_3 limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q index a69fd084f46b..3f818b18534a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q @@ -43,14 +43,14 @@ select ,cast (cint as float) ,cast (cdouble as float) -- to timestamp - ,timestamp_seconds (ctinyint) - ,timestamp_seconds (csmallint) - ,timestamp_seconds (cint) - ,timestamp_seconds (cbigint) - ,timestamp_seconds (cfloat) - ,timestamp_seconds (cdouble) + ,cast (ctinyint as timestamp) + ,cast (csmallint as timestamp) + ,cast (cint as timestamp) + ,cast (cbigint as timestamp) + ,cast (cfloat as timestamp) + ,cast (cdouble as timestamp) ,cast (cboolean1 as timestamp) - ,timestamp_seconds (cbigint * 0) + ,cast (cbigint * 0 as timestamp) ,cast (ctimestamp1 as timestamp) ,cast (cstring1 as timestamp) ,cast (substr(cstring1, 1, 1) as timestamp) @@ -115,14 +115,14 @@ select ,cast (cint as float) ,cast (cdouble as float) -- to timestamp - ,timestamp_seconds (ctinyint) - ,timestamp_seconds (csmallint) - ,timestamp_seconds (cint) - ,timestamp_seconds (cbigint) - ,timestamp_seconds (cfloat) - ,timestamp_seconds (cdouble) + ,cast (ctinyint as timestamp) + ,cast (csmallint as timestamp) + ,cast (cint as timestamp) + ,cast (cbigint as timestamp) + ,cast (cfloat as timestamp) + ,cast (cdouble as timestamp) ,cast (cboolean1 as timestamp) - ,timestamp_seconds (cbigint * 0) + ,cast (cbigint * 0 as timestamp) ,cast (ctimestamp1 as timestamp) ,cast (cstring1 as timestamp) ,cast (substr(cstring1, 1, 1) as timestamp) 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 a5ba0f90b29b..447618a9587e 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 @@ -50,6 +50,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd import org.apache.spark.sql.hive.test.TestHive.implicits._ private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled + private val originalLegacyAllowCastNumericToTimestamp = + TestHive.conf.legacyAllowCastNumericToTimestamp def spark: SparkSession = sparkSession @@ -58,6 +60,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd TestHive.setCacheTables(true) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) } override def afterAll(): Unit = { @@ -65,6 +68,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd TestHive.setCacheTables(false) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + originalLegacyAllowCastNumericToTimestamp) } finally { super.afterAll() } @@ -201,13 +206,13 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), timestamp_seconds(1)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), timestamp_seconds(1)) AS COL20") + val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20") .collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } @@ -552,34 +557,36 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd // Jdk version leads to different query output for double, so not use createQueryTest here test("timestamp cast #1") { - val res = sql("SELECT CAST(timestamp_seconds(1) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head assert(1 == res.getDouble(0)) } + createQueryTest("timestamp cast #2", - "SELECT CAST(timestamp_seconds(1.2) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") test("timestamp cast #3") { - val res = sql("SELECT CAST(timestamp_seconds(1200) AS INT) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } + createQueryTest("timestamp cast #4", - "SELECT CAST(timestamp_seconds(1.2) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") test("timestamp cast #5") { - val res = sql("SELECT CAST(timestamp_seconds(-1) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head assert(-1 == res.get(0)) } createQueryTest("timestamp cast #6", - "SELECT CAST(timestamp_seconds(-1.2) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") test("timestamp cast #7") { - val res = sql("SELECT CAST(timestamp_seconds(-1200) AS INT) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } createQueryTest("timestamp cast #8", - "SELECT CAST(timestamp_seconds(-1.2) AS DOUBLE) FROM src LIMIT 1") + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") From b43a140da1bb7324e75fdf6d9b77563dffededc5 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 9 Jun 2020 15:56:21 +0800 Subject: [PATCH 48/75] remove useless space --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 1 - 1 file changed, 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 b315b996085f..cf3bbdbabc1b 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 @@ -3188,7 +3188,6 @@ class SQLConf extends Serializable with Logging { def legacyAllowCastNumericToTimestamp: Boolean = getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ From 1d5c750557d397619c8bbc08baf47125ecae17c5 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 10 Jun 2020 01:42:32 +0800 Subject: [PATCH 49/75] fix code style.etc --- python/pyspark/sql/functions.py | 2 +- .../src/main/scala/org/apache/spark/sql/functions.scala | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 7ca2ed9cde40..0c8c34dd8799 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1433,7 +1433,7 @@ def timestamp_seconds(col): >>> from pyspark.sql.functions import timestamp_seconds >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() - [Row(ts=datetime.datetime(2008, 12, 25, 07, 30))] + [Row(ts=datetime.datetime(2008, 12, 25, 7, 30))] """ sc = SparkContext._active_spark_context 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 0a3af5111cd1..9b0b5997a792 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 @@ -3363,10 +3363,8 @@ object functions { * @group = "datetime_funcs", * @since = "3.1.0") */ - def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) } - - def array_contains1(column: Column, value: Any): Column = withExpr { - ArrayContains(column.expr, lit(value).expr) + def timestamp_seconds(e: Column): Column = withExpr { + SecondsToTimestamp(e.expr) } ////////////////////////////////////////////////////////////////////////////////////////////// From 7b26d0cf17df1329a9d82ec7f392feec2f8efe90 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 10 Jun 2020 15:19:20 +0800 Subject: [PATCH 50/75] remove duplicated test cases --- python/pyspark/sql/dataframe.py | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 37 ------------------- 2 files changed, 1 insertion(+), 38 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 237e8ad794fa..7bad6964c76f 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -535,7 +535,7 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving >>> from pyspark.sql.functions import timestamp_seconds - >>> sdf.select('name', timestamp_seconds($"time")).withWatermark('time', '10 minutes') + >>> sdf.select('name', timestamp_seconds(sdf.time).alias("time")).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: 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 334d80d1f1ac..7537334be419 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 @@ -252,10 +252,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) } - - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { - assert(!cast(cast(1.toDouble, TimestampType), DoubleType).resolved) - } } test("cast from string") { @@ -336,20 +332,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { 5.toShort) } - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { - assert(!cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType).resolved) - - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - null) - - assert(!cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), - ByteType), TimestampType), LongType), StringType), ShortType).resolved) - - } - checkEvaluation(cast("23", DoubleType), 23d) checkEvaluation(cast("23", IntegerType), 23) checkEvaluation(cast("23", FloatType), 23f) @@ -435,20 +417,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(Float.NaN, TimestampType), null) checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) } - - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { - assert(!cast(cast(tss, ShortType), TimestampType).resolved) - assert(!cast(cast(tss, IntegerType), TimestampType).resolved) - assert(!cast(cast(tss, LongType), TimestampType).resolved) - assert(!cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType).resolved) - assert(!cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType).resolved) - assert(!cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT).resolved) - assert(!cast(cast(0.000001, TimestampType), DoubleType).resolved) - assert(!cast(Double.NaN, TimestampType).resolved) - assert(!cast(1.0 / 0.0, TimestampType).resolved) - assert(!cast(Float.NaN, TimestampType).resolved) - assert(!cast(1.0f / 0.0f, TimestampType).resolved) - } } test("cast from array") { @@ -1082,11 +1050,6 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) } - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { - assert(!cast(cast(1000, TimestampType), LongType).resolved) - assert(!cast(cast(-1200, TimestampType), LongType).resolved) - } - checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) checkEvaluation(cast(123, DecimalType(3, 1)), null) From c7f2a9b323c5354c5dab1354c9a9bda19274dcdc Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 10 Jun 2020 16:06:47 +0800 Subject: [PATCH 51/75] fix python style --- python/pyspark/sql/dataframe.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 7bad6964c76f..593316bf67fc 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -535,7 +535,8 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving >>> from pyspark.sql.functions import timestamp_seconds - >>> sdf.select('name', timestamp_seconds(sdf.time).alias("time")).withWatermark('time', '10 minutes') + >>> sdf.select('name', timestamp_seconds(sdf.time).alias("time")) + ... .withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 8a5e9be31d0993271e4c3453ba04c7e8fdf86147 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 10 Jun 2020 17:50:50 +0800 Subject: [PATCH 52/75] fix code style --- .../spark/sql/catalyst/expressions/CastSuite.scala | 8 -------- .../main/scala/org/apache/spark/sql/functions.scala | 10 +++++----- .../src/test/resources/spark-version-info.properties | 6 ++++++ 3 files changed, 11 insertions(+), 13 deletions(-) create mode 100644 sql/hive/src/test/resources/spark-version-info.properties 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 7537334be419..35b401798013 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 @@ -53,14 +53,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) } - - withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "false") { - if (from.isInstanceOf[NumericType] && to.isInstanceOf[TimestampType]) { - assert(!cast(Literal.create(null, from), to, UTC_OPT).resolved) - } else { - checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) - } - } } test("null cast") { 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 9b0b5997a792..62ad5ea9b593 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 @@ -3358,11 +3358,11 @@ object functions { window(timeColumn, windowDuration, windowDuration, "0 second") } - /** - * Creates timestamp from the number of seconds since UTC epoch.", - * @group = "datetime_funcs", - * @since = "3.1.0") - */ + /** + * Creates timestamp from the number of seconds since UTC epoch. + * @group = datetime_funcs + * @since = 3.1.0 + */ def timestamp_seconds(e: Column): Column = withExpr { SecondsToTimestamp(e.expr) } diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties new file mode 100644 index 000000000000..69db78bdf332 --- /dev/null +++ b/sql/hive/src/test/resources/spark-version-info.properties @@ -0,0 +1,6 @@ +version=3.1.0 +user=dcadmin +revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f +branch=31710-fix-compatibility +date=2020-06-03T13:50:55Z +url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From a6a9bd431fa401be36173a2866f6d56138472f2d Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Wed, 10 Jun 2020 22:55:56 +0800 Subject: [PATCH 53/75] fix failed test case --- ... testing-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...esting-1-237a6af90a857da1efcbe98f6bbbf9d6} | 0 ... cast #2-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...ast #2-1-732ed232ac592c5e7f7c913a88874fd2} | 0 ... cast #4-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...ast #4-1-732ed232ac592c5e7f7c913a88874fd2} | 0 ... cast #6-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...ast #6-1-6d2da5cfada03605834e38bc4075bc79} | 0 ... cast #8-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...ast #8-1-6d2da5cfada03605834e38bc4075bc79} | 0 .../test/queries/clientpositive/decimal_1.q | 1 + .../resources/spark-version-info.properties | 6 --- .../sql/hive/execution/HiveQuerySuite.scala | 43 ++++++++++++------- 13 files changed, 29 insertions(+), 21 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/constant null testing-0-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 => constant null testing-1-237a6af90a857da1efcbe98f6bbbf9d6} (100%) create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #2-0-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 => timestamp cast #2-1-732ed232ac592c5e7f7c913a88874fd2} (100%) create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 => timestamp cast #4-1-732ed232ac592c5e7f7c913a88874fd2} (100%) create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #6-0-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 => timestamp cast #6-1-6d2da5cfada03605834e38bc4075bc79} (100%) create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #8-0-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 => timestamp cast #8-1-6d2da5cfada03605834e38bc4075bc79} (100%) delete mode 100644 sql/hive/src/test/resources/spark-version-info.properties diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/constant null testing-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 b/sql/hive/src/test/resources/golden/constant null testing-1-237a6af90a857da1efcbe98f6bbbf9d6 similarity index 100% rename from sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 rename to sql/hive/src/test/resources/golden/constant null testing-1-237a6af90a857da1efcbe98f6bbbf9d6 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #2-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #2-1-732ed232ac592c5e7f7c913a88874fd2 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 rename to sql/hive/src/test/resources/golden/timestamp cast #2-1-732ed232ac592c5e7f7c913a88874fd2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #4-1-732ed232ac592c5e7f7c913a88874fd2 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 rename to sql/hive/src/test/resources/golden/timestamp cast #4-1-732ed232ac592c5e7f7c913a88874fd2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #6-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #6-1-6d2da5cfada03605834e38bc4075bc79 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 rename to sql/hive/src/test/resources/golden/timestamp cast #6-1-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #8-1-6d2da5cfada03605834e38bc4075bc79 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 rename to sql/hive/src/test/resources/golden/timestamp cast #8-1-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q index f52b1923eb06..7ccfde4bcdcf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q @@ -1,4 +1,5 @@ set hive.fetch.task.conversion=more; +set spark.sql.legacy.allowCastNumericToTimestamp=true; drop table if exists decimal_1; diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/sql/hive/src/test/resources/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file 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 447618a9587e..b0506af96216 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 @@ -50,8 +50,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd import org.apache.spark.sql.hive.test.TestHive.implicits._ private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled - private val originalLegacyAllowCastNumericToTimestamp = - TestHive.conf.legacyAllowCastNumericToTimestamp +// private val originalLegacyAllowCastNumericToTimestamp = +// TestHive.conf.legacyAllowCastNumericToTimestamp def spark: SparkSession = sparkSession @@ -60,7 +60,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd TestHive.setCacheTables(true) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) - TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) +// TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) } override def afterAll(): Unit = { @@ -68,8 +68,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd TestHive.setCacheTables(false) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) - TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - originalLegacyAllowCastNumericToTimestamp) +// TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, +// originalLegacyAllowCastNumericToTimestamp) } finally { super.afterAll() } @@ -185,7 +185,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd "SELECT CAST(CAST('NaN' AS DOUBLE) AS DECIMAL(1,1)) FROM src LIMIT 1") createQueryTest("constant null testing", - """SELECT + """set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT |IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL1, |IF(TRUE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2, |IF(FALSE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL3, @@ -212,7 +213,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |FROM src LIMIT 1""".stripMargin) test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20") + val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), TIMESTAMP_SECONDS(1)) AS COL20") .collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } @@ -557,36 +558,48 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd // Jdk version leads to different query output for double, so not use createQueryTest here test("timestamp cast #1") { - val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(1 == res.getDouble(0)) } createQueryTest("timestamp cast #2", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + """ + |set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 + """.stripMargin) test("timestamp cast #3") { - val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } createQueryTest("timestamp cast #4", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + """ + |set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 + """.stripMargin) test("timestamp cast #5") { - val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(-1 == res.get(0)) } createQueryTest("timestamp cast #6", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + """ + |set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 + """.stripMargin) test("timestamp cast #7") { - val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1200) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } createQueryTest("timestamp cast #8", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + """ + |set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 + """.stripMargin) createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") From f995f46a5eb547811d7f235810b5060d80862577 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 11 Jun 2020 13:44:10 +0800 Subject: [PATCH 54/75] rebase --- python/pyspark/resource/spark-version-info.properties | 6 ++++++ sql/core/src/test/resources/spark-version-info.properties | 6 ++++++ 2 files changed, 12 insertions(+) create mode 100644 python/pyspark/resource/spark-version-info.properties create mode 100644 sql/core/src/test/resources/spark-version-info.properties diff --git a/python/pyspark/resource/spark-version-info.properties b/python/pyspark/resource/spark-version-info.properties new file mode 100644 index 000000000000..69db78bdf332 --- /dev/null +++ b/python/pyspark/resource/spark-version-info.properties @@ -0,0 +1,6 @@ +version=3.1.0 +user=dcadmin +revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f +branch=31710-fix-compatibility +date=2020-06-03T13:50:55Z +url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file diff --git a/sql/core/src/test/resources/spark-version-info.properties b/sql/core/src/test/resources/spark-version-info.properties new file mode 100644 index 000000000000..69db78bdf332 --- /dev/null +++ b/sql/core/src/test/resources/spark-version-info.properties @@ -0,0 +1,6 @@ +version=3.1.0 +user=dcadmin +revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f +branch=31710-fix-compatibility +date=2020-06-03T13:50:55Z +url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From c8d5aa5cf1c0c5eaf85ad6e01b008f025e468d55 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 11 Jun 2020 13:44:24 +0800 Subject: [PATCH 55/75] rebase --- sql/core/src/test/resources/spark-version-info.properties | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 sql/core/src/test/resources/spark-version-info.properties diff --git a/sql/core/src/test/resources/spark-version-info.properties b/sql/core/src/test/resources/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/sql/core/src/test/resources/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From b4f4d537d95ded13e80e8171036e0d36410e9732 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 11 Jun 2020 14:14:46 +0800 Subject: [PATCH 56/75] delete test file --- python/pyspark/resource/spark-version-info.properties | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 python/pyspark/resource/spark-version-info.properties diff --git a/python/pyspark/resource/spark-version-info.properties b/python/pyspark/resource/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/python/pyspark/resource/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From 7132fca3185a16790655046cf016db639e7edfb1 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Thu, 11 Jun 2020 22:15:33 +0800 Subject: [PATCH 57/75] rebase lastet code --- sql/hive/src/test/resources/spark-version-info.properties | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 sql/hive/src/test/resources/spark-version-info.properties diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/sql/hive/src/test/resources/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From 7ffcde2ff8cf482ea13bc3f782da059099383a59 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 12 Jun 2020 11:46:14 +0800 Subject: [PATCH 58/75] fix error cases --- core/pom.xml | 2 +- project/MimaBuild.scala | 99 - project/MimaExcludes.scala | 1698 ----------------- project/SparkBuild.scala | 1058 ---------- project/build.properties | 17 - project/plugins.sbt | 65 - .../execution/HiveCompatibilitySuite.scala | 6 +- ...ecimal_1-1-dce645ca43d2bf888c4d75c456e42f3 | 1 + ...mal_1-10-e4e90927ac59f5920de3dc61c3288dde} | 0 ...mal_1-11-c20dea9d716bef1bdbdef71323b1cc5b} | 0 ...mal_1-12-f2f975b73220512d4bf2b9bd93354aba} | 0 ...mal_1-13-2c2325880ea79c8e308398d46c8565f8} | 0 ...mal_1-14-c4c33bdb9f3c6cad77552f0f353092d3} | 0 ...mal_1-15-e45935cfffb9045394e804d0d1fc52f0} | 0 ...mal_1-16-31ecaab3afa056fcc656d6e54f845cf4} | 0 ...imal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972} | 0 ...cimal_1-3-ee665100ca5de3a006df43e97cfa707} | 0 ...imal_1-4-80fc87cab17ceffea334afbb230a6653} | 0 ...imal_1-5-5dd925bba25f735bfd6442a841afe119} | 0 ...cimal_1-6-bfab296ca5693e647e33899dfeeb256} | 0 ...imal_1-7-a402201ed5159941384d40e09dc367a5} | 0 ...cimal_1-8-2cfd7d00bc37a8e433ad005896173c1} | 0 ...imal_1-9-84cd75e494d113a48c4145298177d6d8} | 0 .../hive/execution/HiveComparisonTest.scala | 1 + 24 files changed, 4 insertions(+), 2943 deletions(-) delete mode 100644 project/MimaBuild.scala delete mode 100644 project/MimaExcludes.scala delete mode 100644 project/SparkBuild.scala delete mode 100644 project/build.properties delete mode 100644 project/plugins.sbt create mode 100644 sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b => decimal_1-10-e4e90927ac59f5920de3dc61c3288dde} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 => decimal_1-11-c20dea9d716bef1bdbdef71323b1cc5b} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-11-f2f975b73220512d4bf2b9bd93354aba => decimal_1-12-f2f975b73220512d4bf2b9bd93354aba} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-12-2c2325880ea79c8e308398d46c8565f8 => decimal_1-13-2c2325880ea79c8e308398d46c8565f8} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 => decimal_1-14-c4c33bdb9f3c6cad77552f0f353092d3} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 => decimal_1-15-e45935cfffb9045394e804d0d1fc52f0} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 => decimal_1-16-31ecaab3afa056fcc656d6e54f845cf4} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 => decimal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-2-ee665100ca5de3a006df43e97cfa707 => decimal_1-3-ee665100ca5de3a006df43e97cfa707} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-3-80fc87cab17ceffea334afbb230a6653 => decimal_1-4-80fc87cab17ceffea334afbb230a6653} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-4-5dd925bba25f735bfd6442a841afe119 => decimal_1-5-5dd925bba25f735bfd6442a841afe119} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-5-bfab296ca5693e647e33899dfeeb256 => decimal_1-6-bfab296ca5693e647e33899dfeeb256} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-6-a402201ed5159941384d40e09dc367a5 => decimal_1-7-a402201ed5159941384d40e09dc367a5} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-8-84cd75e494d113a48c4145298177d6d8 => decimal_1-8-2cfd7d00bc37a8e433ad005896173c1} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-9-e4e90927ac59f5920de3dc61c3288dde => decimal_1-9-84cd75e494d113a48c4145298177d6d8} (100%) diff --git a/core/pom.xml b/core/pom.xml index b0f68880f1d8..39da57dd954a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -476,7 +476,7 @@ - + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala deleted file mode 100644 index 10c02103aedd..000000000000 --- a/project/MimaBuild.scala +++ /dev/null @@ -1,99 +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 sbt._ -import sbt.Keys.version - -import com.typesafe.tools.mima.core._ -import com.typesafe.tools.mima.core.MissingClassProblem -import com.typesafe.tools.mima.core.MissingTypesProblem -import com.typesafe.tools.mima.core.ProblemFilters._ -import com.typesafe.tools.mima.plugin.MimaKeys.{mimaBinaryIssueFilters, mimaPreviousArtifacts} -import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings - - -object MimaBuild { - - def excludeMember(fullName: String) = Seq( - ProblemFilters.exclude[MissingMethodProblem](fullName), - // Sometimes excluded methods have default arguments and - // they are translated into public methods/fields($default$) in generated - // bytecode. It is not possible to exhaustively list everything. - // But this should be okay. - ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$2"), - ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$1"), - ProblemFilters.exclude[MissingFieldProblem](fullName), - ProblemFilters.exclude[IncompatibleResultTypeProblem](fullName), - ProblemFilters.exclude[IncompatibleMethTypeProblem](fullName), - ProblemFilters.exclude[IncompatibleFieldTypeProblem](fullName) - ) - - // Exclude a single class - def excludeClass(className: String) = Seq( - ProblemFilters.exclude[Problem](className + ".*"), - ProblemFilters.exclude[MissingClassProblem](className), - ProblemFilters.exclude[MissingTypesProblem](className) - ) - - // Exclude a Spark class, that is in the package org.apache.spark - def excludeSparkClass(className: String) = { - excludeClass("org.apache.spark." + className) - } - - // Exclude a Spark package, that is in the package org.apache.spark - def excludeSparkPackage(packageName: String) = { - ProblemFilters.exclude[Problem]("org.apache.spark." + packageName + ".*") - } - - def ignoredABIProblems(base: File, currentSparkVersion: String) = { - - // Excludes placed here will be used for all Spark versions - val defaultExcludes = Seq() - - // Read package-private excludes from file - val classExcludeFilePath = file(base.getAbsolutePath + "/.generated-mima-class-excludes") - val memberExcludeFilePath = file(base.getAbsolutePath + "/.generated-mima-member-excludes") - - val ignoredClasses: Seq[String] = - if (!classExcludeFilePath.exists()) { - Seq() - } else { - IO.read(classExcludeFilePath).split("\n") - } - - val ignoredMembers: Seq[String] = - if (!memberExcludeFilePath.exists()) { - Seq() - } else { - IO.read(memberExcludeFilePath).split("\n") - } - - defaultExcludes ++ ignoredClasses.flatMap(excludeClass) ++ - ignoredMembers.flatMap(excludeMember) ++ MimaExcludes.excludes(currentSparkVersion) - } - - def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { - val organization = "org.apache.spark" - val previousSparkVersion = "2.4.0" - val project = projectRef.project - val fullId = "spark-" + project + "_2.12" - mimaDefaultSettings ++ - Seq(mimaPreviousArtifacts := Set(organization % fullId % previousSparkVersion), - mimaBinaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value)) - } - -} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala deleted file mode 100644 index 57fbb125dc47..000000000000 --- a/project/MimaExcludes.scala +++ /dev/null @@ -1,1698 +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 com.typesafe.tools.mima.core._ -import com.typesafe.tools.mima.core.ProblemFilters._ - -/** - * Additional excludes for checking of Spark's binary compatibility. - * - * This acts as an official audit of cases where we excluded other classes. Please use the narrowest - * possible exclude here. MIMA will usually tell you what exclude to use, e.g.: - * - * ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.take") - * - * It is also possible to exclude Spark classes and packages. This should be used sparingly: - * - * MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") - * - * For a new Spark version, please update MimaBuild.scala to reflect the previous version. - */ -object MimaExcludes { - - // Exclude rules for 3.1.x - lazy val v31excludes = v30excludes ++ Seq( - // [SPARK-31077] Remove ChiSqSelector dependency on mllib.ChiSqSelectorModel - // private constructor - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.this"), - // [SPARK-31127] Implement abstract Selector - // org.apache.spark.ml.feature.ChiSqSelectorModel type hierarchy change - // before: class ChiSqSelector extends Estimator with ChiSqSelectorParams - // after: class ChiSqSelector extends PSelector - // false positive, no binary incompatibility - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelector"), - - //[SPARK-31840] Add instance weight support in LogisticRegressionSummary - // weightCol in org.apache.spark.ml.classification.LogisticRegressionSummary is present only in current version - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightCol") - ) - - // 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"), - - // [SPARK-28980][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 - 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"), - - // [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"), - - // [SPARK-26580][SQL][ML][FOLLOW-UP] Throw exception when use untyped UDF by default - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.UnaryTransformer.this"), - - // [SPARK-27090][CORE] Removing old LEGACY_DRIVER_IDENTIFIER ("") - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.LEGACY_DRIVER_IDENTIFIER"), - - // [SPARK-25838] Remove formatVersion from Saveable - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.DistributedLDAModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.LocalLDAModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeansModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.PowerIterationClusteringModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.GaussianMixtureModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.recommendation.MatrixFactorizationModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.ChiSqSelectorModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.Word2VecModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.SVMModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.LogisticRegressionModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.NaiveBayesModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.Saveable.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.fpm.FPGrowthModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.fpm.PrefixSpanModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.IsotonicRegressionModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.RidgeRegressionModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.LassoModel.formatVersion"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.LinearRegressionModel.formatVersion"), - - // [SPARK-26132] Remove support for Scala 2.11 in Spark 3.0.0 - ProblemFilters.exclude[DirectAbstractMethodProblem]("scala.concurrent.Future.transformWith"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("scala.concurrent.Future.transform"), - - // [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core. - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.deploy.security.HiveDelegationTokenProvider"), - - // [SPARK-26329][CORE] Faster polling of executor memory metrics. - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.apply"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.copy$default$6"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.this"), - - // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), - 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"), - - // [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"), - - // [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.launcher.SparkAppHandle.getError"), - - // [SPARK-25867] Remove KMeans computeCost - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), - - // [SPARK-26127] Remove deprecated setters from tree regression and classification models - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setSeed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMinInfoGain"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setCacheNodeIds"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setCheckpointInterval"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMaxDepth"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setImpurity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMaxMemoryInMB"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMaxBins"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMinInstancesPerNode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setSeed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMinInfoGain"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setSubsamplingRate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxIter"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setCacheNodeIds"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setCheckpointInterval"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxDepth"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setImpurity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxMemoryInMB"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setStepSize"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxBins"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMinInstancesPerNode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setFeatureSubsetStrategy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setSeed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMinInfoGain"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setSubsamplingRate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setCacheNodeIds"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setCheckpointInterval"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMaxDepth"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setImpurity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMaxMemoryInMB"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setFeatureSubsetStrategy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMaxBins"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMinInstancesPerNode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setNumTrees"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setSeed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMinInfoGain"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setCacheNodeIds"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setCheckpointInterval"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMaxDepth"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setImpurity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMaxMemoryInMB"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMaxBins"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMinInstancesPerNode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setSeed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMinInfoGain"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setSubsamplingRate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxIter"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setCacheNodeIds"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setCheckpointInterval"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxDepth"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setImpurity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxMemoryInMB"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setStepSize"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxBins"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMinInstancesPerNode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setFeatureSubsetStrategy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setSeed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMinInfoGain"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setSubsamplingRate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setCacheNodeIds"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setCheckpointInterval"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMaxDepth"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setImpurity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMaxMemoryInMB"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMaxBins"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMinInstancesPerNode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setNumTrees"), - - // [SPARK-26090] Resolve most miscellaneous deprecation and build warnings for Spark 3 - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.stat.test.BinarySampleBeanInfo"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.LabeledPointBeanInfo"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.LabeledPointBeanInfo"), - - // [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-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"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleBytesWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleWriteTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.fMeasure"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.recall"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.precision"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.util.MLWriter.context"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.util.MLReader.context"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.util.GeneralMLWriter.context"), - - // [SPARK-25737] Remove JavaSparkContextVarargsWorkaround - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.api.java.JavaSparkContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.union"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.union"), - - // [SPARK-16775] Remove deprecated accumulator v1 APIs - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulable"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulator"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulator$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulableParam"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$FloatAccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$DoubleAccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$LongAccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$IntAccumulatorParam$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulable"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulableCollection"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulator"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.LegacyAccumulatorWrapper"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.intAccumulator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.accumulable"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.doubleAccumulator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.accumulator"), - - // [SPARK-24109] Remove class SnappyOutputStreamWrapper - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.SnappyCompressionCodec.version"), - - // [SPARK-19287] JavaPairRDD flatMapValues requires function returning Iterable, not Iterator - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.api.java.JavaPairRDD.flatMapValues"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaPairDStream.flatMapValues"), - - // [SPARK-25680] SQL execution listener shouldn't happen on execution thread - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.util.ExecutionListenerManager.clone"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.util.ExecutionListenerManager.this"), - - // [SPARK-25862][SQL] Remove rangeBetween APIs introduced in SPARK-21608 - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.unboundedFollowing"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.unboundedPreceding"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.currentRow"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.expressions.Window.rangeBetween"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.expressions.WindowSpec.rangeBetween"), - - // [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.nextCredentialRenewalTime"), - - // [SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.OneHotEncoder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator$"), - - // [SPARK-30329][ML] add iterator/foreach methods for Vectors - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Vector.activeIterator"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.activeIterator"), - - // [SPARK-26141] Enable custom metrics implementation in shuffle write - // Following are Java private classes - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.TimeTrackingOutputStream.this"), - - // [SPARK-26139] Implement shuffle write metrics in SQL - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ShuffleDependency.this"), - - // [SPARK-26362][CORE] Remove 'spark.driver.allowMultipleContexts' to disallow multiple creation of SparkContexts - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.setActiveContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.markPartiallyConstructed"), - - // [SPARK-26457] Show hadoop configurations in HistoryServer environment tab - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this"), - - // [SPARK-30144][ML] Make MultilayerPerceptronClassificationModel extend MultilayerPerceptronParams - 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) => - !cls.fullName.startsWith("org.apache.spark.sql.sources.v2") - case _ => true - }, - - // [SPARK-27521][SQL] Move data source v2 to catalyst module - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarBatch"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ArrowColumnVector"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarRow"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarArray"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarMap"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnVector"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThanOrEqual"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringEndsWith"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThanOrEqual$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.In$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Not"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNotNull"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThan"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThanOrEqual"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualNullSafe$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThan$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.In"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.And"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringStartsWith$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualNullSafe"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringEndsWith$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThanOrEqual$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Not$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNull$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThan$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNotNull$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Or"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualTo$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThan"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringContains"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Filter"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNull"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualTo"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.And$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Or$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringStartsWith"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringContains$"), - - // [SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction) - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction$"), - ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.inputTypes"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullableTypes_="), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.dataType"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.f"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.this"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNonNullable"), - ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNonNullable"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullable"), - ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullable"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNondeterministic"), - ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNondeterministic"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.deterministic"), - ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.deterministic"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.apply"), - ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.apply"), - ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.withName"), - ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.withName"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productElement"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productArity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$2"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.canEqual"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$1"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productIterator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productPrefix"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$3"), - - // [SPARK-11215][ML] Add multiple columns support to StringIndexer - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.StringIndexer.validateAndTransformSchema"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.StringIndexerModel.validateAndTransformSchema"), - - // [SPARK-26616][MLlib] Expose document frequency in IDFModel - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"), - - // [SPARK-28199][SS] Remove deprecated ProcessingTime - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime$"), - - // [SPARK-25382][SQL][PYSPARK] Remove ImageSchema.readImages in 3.0 - 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"), - - // [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"), - - // [SPARK-29348] Add observable metrics. - 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"), - - // [SPARK-29543][SS][UI] Init structured streaming ui - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStartedEvent.this"), - - // [SPARK-30667][CORE] Add allGather method to BarrierTaskContext - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.RequestToSync") - ) - - // Exclude rules for 2.4.x - lazy val v24excludes = v23excludes ++ Seq( - // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$"), - - // [SPARK-25248] add package private methods to TaskContext - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markTaskFailed"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markInterrupted"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.fetchFailed"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markTaskCompleted"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getLocalProperties"), - - // [SPARK-10697][ML] Add lift to Association rules - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.fpm.FPGrowthModel.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.fpm.AssociationRules#Rule.this"), - - // [SPARK-24296][CORE] Replicate large blocks as a stream. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockRpcServer.this"), - // [SPARK-23528] Add numIter to ClusteringSummary - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.ClusteringSummary.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.KMeansSummary.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.BisectingKMeansSummary.this"), - // [SPARK-6237][NETWORK] Network-layer changes to allow stream upload - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockRpcServer.receive"), - - // [SPARK-20087][CORE] Attach accumulators / metrics to 'TaskKilled' end reason - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.this"), - - // [SPARK-22941][core] Do not exit JVM when submit fails with in-process launcher. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.printWarning"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.parseSparkConfProperty"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.printVersionAndExit"), - - // [SPARK-23412][ML] Add cosine distance measure to BisectingKmeans - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasDistanceMeasure.org$apache$spark$ml$param$shared$HasDistanceMeasure$_setter_$distanceMeasure_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasDistanceMeasure.getDistanceMeasure"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasDistanceMeasure.distanceMeasure"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel#SaveLoadV1_0.load"), - - // [SPARK-20659] Remove StorageStatus, or make it private - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.totalOffHeapStorageMemory"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.usedOffHeapStorageMemory"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.usedOnHeapStorageMemory"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.totalOnHeapStorageMemory"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.getExecutorStorageStatus"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.numBlocks"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.numRddBlocks"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.containsBlock"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.rddBlocksById"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.numRddBlocksById"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.memUsedByRdd"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.cacheSize"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.rddStorageLevel"), - - // [SPARK-23455][ML] Default Params in ML should be saved separately in metadata - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.paramMap"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$paramMap_="), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.defaultParamMap"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$defaultParamMap_="), - - // [SPARK-7132][ML] Add fit with validation set to spark.ml GBT - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), - - // [SPARK-23042] Use OneHotEncoderModel to encode labels in MultilayerPerceptronClassifier - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.classification.LabelConverter"), - - // [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getDateOfNextUpdate"), - - // [SPARK-23366] Improve hot reading path in ReadAheadInputStream - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.ReadAheadInputStream.this"), - - // [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.addJarToClasspath"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.mergeFileLists"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment$default$2"), - - // Data Source V2 API changes - // TODO: they are unstable APIs and should not be tracked by mima. - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.ReadSupportWithSchema"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.createDataReaderFactories"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.createBatchDataReaderFactories"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.planBatchInputPartitions"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanUnsafeRow"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader.createDataReaderFactories"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader.planInputPartitions"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsPushDownCatalystFilters"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataReader"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.getStatistics"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.estimateStatistics"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataReaderFactory"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.ContinuousDataReader"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.SupportsWriteInternalRow"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createDataWriter"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createDataWriter"), - - // Changes to HasRawPredictionCol. - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.rawPredictionCol"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.org$apache$spark$ml$param$shared$HasRawPredictionCol$_setter_$rawPredictionCol_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.getRawPredictionCol"), - - // [SPARK-15526][ML][FOLLOWUP] Make JPMML provided scope to avoid including unshaded JARs - (problem: Problem) => problem match { - case MissingClassProblem(cls) => - !cls.fullName.startsWith("org.sparkproject.jpmml") && - !cls.fullName.startsWith("org.sparkproject.dmg.pmml") && - !cls.fullName.startsWith("org.spark_project.jpmml") && - !cls.fullName.startsWith("org.spark_project.dmg.pmml") - case _ => true - } - ) - - // Exclude rules for 2.3.x - lazy val v23excludes = v22excludes ++ Seq( - // [SPARK-22897] Expose stageAttemptId in TaskContext - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.stageAttemptNumber"), - - // SPARK-22789: Map-only continuous processing execution - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.startQuery$default$8"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.startQuery$default$6"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.startQuery$default$9"), - - // SPARK-22372: Make cluster submission use SparkApplication. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getSecretKeyFromUserCredentials"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.isYarnMode"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getCurrentUserCredentials"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.addSecretKeyToUserCredentials"), - - // SPARK-18085: Better History Server scalability for many / large applications - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ExecutorSummary.executorLogs"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.getSparkUI"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.env.EnvironmentListener"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.exec.ExecutorsListener"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.storage.StorageListener"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.StorageStatusListener"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorStageSummary.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.JobData.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkStatusTracker.this"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.jobs.JobProgressListener"), - - // [SPARK-20495][SQL] Add StorageLevel to cacheTable API - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.cacheTable"), - - // [SPARK-19937] Add remote bytes read to disk. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetricDistributions.this"), - - // [SPARK-21276] Update lz4-java to the latest (v1.4.0) - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.io.LZ4BlockInputStream"), - - // [SPARK-17139] Add model summary for MultinomialLogisticRegression - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictionCol"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.labels"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.truePositiveRateByLabel"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.falsePositiveRateByLabel"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.precisionByLabel"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.recallByLabel"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.fMeasureByLabel"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.accuracy"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedTruePositiveRate"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedFalsePositiveRate"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedRecall"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedPrecision"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedFMeasure"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.asBinary"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.org$apache$spark$ml$classification$LogisticRegressionSummary$$multiclassMetrics"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.org$apache$spark$ml$classification$LogisticRegressionSummary$_setter_$org$apache$spark$ml$classification$LogisticRegressionSummary$$multiclassMetrics_="), - - // [SPARK-14280] Support Scala 2.12 - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.FutureAction.transformWith"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.FutureAction.transform"), - - // [SPARK-21087] CrossValidator, TrainValidationSplit expose sub models after fitting: Scala - ProblemFilters.exclude[FinalClassProblem]("org.apache.spark.ml.tuning.CrossValidatorModel$CrossValidatorModelWriter"), - ProblemFilters.exclude[FinalClassProblem]("org.apache.spark.ml.tuning.TrainValidationSplitModel$TrainValidationSplitModelWriter"), - - // [SPARK-21728][CORE] Allow SparkSubmit to use Logging - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.downloadFileList"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.downloadFile"), - - // [SPARK-21714][CORE][YARN] Avoiding re-uploading remote resources in yarn client mode - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment"), - - // [SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0 - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.network.util.AbstractFileRegion.transfered"), - - // [SPARK-20643][CORE] Add listener implementation to collect app state - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$5"), - - // [SPARK-20648][CORE] Port JobsTab and StageTab to the new UI backend - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$12"), - - // [SPARK-21462][SS] Added batchId to StreamingQueryProgress.json - // [SPARK-21409][SS] Expose state store memory usage in SQL metrics and progress updates - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StateOperatorProgress.this"), - - // [SPARK-22278][SS] Expose current event time watermark and current processing time in GroupState - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.GroupState.getCurrentWatermarkMs"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.GroupState.getCurrentProcessingTimeMs"), - - // [SPARK-20542][ML][SQL] Add an API to Bucketizer that can bin multiple columns - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasOutputCols.org$apache$spark$ml$param$shared$HasOutputCols$_setter_$outputCols_="), - - // [SPARK-18619][ML] Make QuantileDiscretizer/Bucketizer/StringIndexer/RFormula inherit from HasHandleInvalid - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.Bucketizer.getHandleInvalid"), - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.StringIndexer.getHandleInvalid"), - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.getHandleInvalid"), - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.StringIndexerModel.getHandleInvalid") - ) - - // Exclude rules for 2.2.x - lazy val v22excludes = v21excludes ++ Seq( - // [SPARK-20355] Add per application spark version on the history server headerpage - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), - - // [SPARK-19652][UI] Do auth checks for REST API access. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.withSparkUI"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.status.api.v1.UIRootFromServletContext"), - - // [SPARK-18663][SQL] Simplify CountMinSketch aggregate implementation - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.CountMinSketch.toByteArray"), - - // [SPARK-18949] [SQL] Add repairTable API to Catalog - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.recoverPartitions"), - - // [SPARK-18537] Add a REST api to spark streaming - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.streaming.scheduler.StreamingListener.onStreamingStarted"), - - // [SPARK-19148][SQL] do not expose the external table concept in Catalog - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createTable"), - - // [SPARK-14272][ML] Add logLikelihood in GaussianMixtureSummary - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.GaussianMixtureSummary.this"), - - // [SPARK-19267] Fetch Failure handling robust to user error handling - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.setFetchFailed"), - - // [SPARK-19069] [CORE] Expose task 'status' and 'duration' in spark history server REST API. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$10"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$11"), - - // [SPARK-17161] Removing Python-friendly constructors not needed - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.OneVsRestModel.this"), - - // [SPARK-19820] Allow reason to be specified to task kill - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.TaskKilled$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productElement"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productArity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.canEqual"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productIterator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.countTowardsTaskFailures"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productPrefix"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.toErrorString"), - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.TaskKilled.toString"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.killTaskIfInterrupted"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getKillReason"), - - // [SPARK-19876] Add one time trigger, and improve Trigger APIs - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.sql.streaming.Trigger"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.streaming.ProcessingTime"), - - // [SPARK-17471][ML] Add compressed method to ML matrices - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.compressed"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.compressedColMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.compressedRowMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.isRowMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.isColMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSparseSizeInBytes"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDense"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparse"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseRowMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseRowMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseColMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getDenseSizeInBytes"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseColMajor"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseMatrix"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseMatrix"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSizeInBytes"), - - // [SPARK-18693] Added weightSum to trait MultivariateStatisticalSummary - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.stat.MultivariateStatisticalSummary.weightSum") - ) ++ Seq( - // [SPARK-17019] Expose on-heap and off-heap memory usage in various places - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.this"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.apply"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.StorageStatus.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.RDDDataDistribution.this") - ) - - // Exclude rules for 2.1.x - lazy val v21excludes = v20excludes ++ { - Seq( - // [SPARK-17671] Spark 2.0 history server summary page is slow even set spark.history.ui.maxApplications - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.deploy.history.HistoryServer.getApplicationList"), - // [SPARK-14743] Improve delegation token handling in secure cluster - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal"), - // [SPARK-16199][SQL] Add a method to list the referenced columns in data source Filter - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.Filter.references"), - // [SPARK-16853][SQL] Fixes encoder error in DataSet typed select - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.select"), - // [SPARK-16967] Move Mesos to Module - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkMasterRegex.MESOS_REGEX"), - // [SPARK-16240] ML persistence backward compatibility for LDA - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.clustering.LDA$"), - // [SPARK-17717] Add Find and Exists method to Catalog. - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getDatabase"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getTable"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getFunction"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.databaseExists"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"), - - // [SPARK-17731][SQL][Streaming] Metrics for structured streaming - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryStarted"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), - - // [SPARK-18516][SQL] Split state and progress in streaming - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.SourceStatus"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.SinkStatus"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.sinkStatus"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.sourceStatuses"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQuery.id"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.lastProgress"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.recentProgress"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.id"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.get"), - - // [SPARK-17338][SQL] add global temp view - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), - - // [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness. - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.aggregationDepth"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.getAggregationDepth"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.org$apache$spark$ml$param$shared$HasAggregationDepth$_setter_$aggregationDepth_="), - - // [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.TaskInfo.accumulables"), - - // [SPARK-18657] Add StreamingQuery.runId - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.runId"), - - // [SPARK-18694] Add StreamingQuery.explain and exception to Python and fix StreamingQueryException - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryException$"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.startOffset"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.endOffset"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryException.query") - ) - } - - // Exclude rules for 2.0.x - lazy val v20excludes = { - Seq( - ProblemFilters.exclude[Problem]("org.apache.spark.rpc.*"), - ProblemFilters.exclude[Problem]("org.spark-project.jetty.*"), - ProblemFilters.exclude[Problem]("org.spark_project.jetty.*"), - ProblemFilters.exclude[Problem]("org.sparkproject.jetty.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.internal.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.unused.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.unsafe.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.memory.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.util.collection.unsafe.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.sql.catalyst.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.sql.execution.*"), - ProblemFilters.exclude[Problem]("org.apache.spark.sql.internal.*"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.status.api.v1.ApplicationAttemptInfo.$default$5"), - // SPARK-14042 Add custom coalescer support - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"), - // SPARK-15532 Remove isRootContext flag from SQLContext. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.isRootContext"), - // SPARK-12600 Remove SQL deprecated methods - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.applySchema"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.parquetFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jdbc"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.load"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.dialectClassName"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.getSQLDialect"), - // SPARK-13664 Replace HadoopFsRelation with FileFormat - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.LibSVMRelation"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelationProvider"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache"), - // SPARK-15543 Rename DefaultSources to make them more self-describing - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.DefaultSource") - ) ++ Seq( - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory"), - // SPARK-14358 SparkListener from trait to abstract class - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.addSparkListener"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.JavaSparkListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.SparkFirehoseListener"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.scheduler.SparkListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.jobs.JobProgressListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.exec.ExecutorsListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.env.EnvironmentListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.storage.StorageListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.StorageStatusListener") - ) ++ - Seq( - // SPARK-3369 Fix Iterable/Iterator in Java API - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapFunction2.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapFunction2.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.PairFlatMapFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.PairFlatMapFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.CoGroupFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.CoGroupFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.MapPartitionsFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.MapPartitionsFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapGroupsFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapGroupsFunction.call") - ) ++ - Seq( - // [SPARK-6429] Implement hashCode and equals together - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Partition.org$apache$spark$Partition$$super=uals") - ) ++ - Seq( - // SPARK-4819 replace Guava Optional - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getCheckpointDir"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getSparkHome"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner") - ) ++ - Seq( - // SPARK-12481 Remove Hadoop 1.x - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.mapred.SparkHadoopMapRedUtil"), - // SPARK-12615 Remove deprecated APIs in core - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.$default$6"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.numericRDDToDoubleRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intToIntWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.writableWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToAsyncRDDActions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.boolToBoolWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longToLongWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToOrderedRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.booleanWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringToText"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleToDoubleWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToSequenceFileRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesToBytesWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatToFloatWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions$default$4"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addOnCompleteCallback"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.runningLocally"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.attemptId"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.defaultMinSplits"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.runJob"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.runJob"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.tachyonFolderName"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.initLocalProperties"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearJars"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearFiles"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith$default$2"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.toArray"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith$default$2"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.filterWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.foreachWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit$default$2"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.SequenceFileRDDFunctions.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.splits"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.toArray"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.defaultMinSplits"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearJars"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearFiles"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.externalBlockStoreFolderName"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockManager"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore") - ) ++ Seq( - // SPARK-12149 Added new fields to ExecutorSummary - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") - ) ++ - // SPARK-12665 Remove deprecated and unused classes - Seq( - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.graphx.GraphKryoRegistrator"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") - ) ++ Seq( - // SPARK-12591 Register OpenHashMapBasedStateMap for Kryo - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoInputDataInputBridge"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoOutputDataOutputBridge") - ) ++ Seq( - // SPARK-12510 Refactor ActorReceiver to support Java - ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") - ) ++ Seq( - // SPARK-12895 Implement TaskMetrics using accumulators - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.internalMetricsToAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectInternalAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectAccumulators") - ) ++ Seq( - // SPARK-12896 Send only accumulator updates to driver, not TaskMetrics - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Accumulator.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.initialValue") - ) ++ Seq( - // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log__="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log__="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log__="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") - ) ++ Seq( - // SPARK-12689 Migrate DDL parsing to the newly absorbed parser - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLParser"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLException"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.ddlParser") - ) ++ Seq( - // SPARK-7799 Add "streaming-akka" project - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$6"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$5"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$4"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$3"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.actorStream"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.streaming.zeromq.ZeroMQReceiver"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver$Supervisor") - ) ++ Seq( - // SPARK-12348 Remove deprecated Streaming APIs. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.dstream.DStream.foreach"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions$default$4"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.awaitTermination"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.networkStream"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.api.java.JavaStreamingContextFactory"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.awaitTermination"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.sc"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreachRDD"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreach"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.getOrCreate") - ) ++ Seq( - // SPARK-12847 Remove StreamingListenerBus and post all Streaming events to the same thread as Spark events - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus") - ) ++ Seq( - // SPARK-11622 Make LibSVMRelation extends HadoopFsRelation and Add LibSVMOutputWriter - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.source.libsvm.DefaultSource"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.source.libsvm.DefaultSource.createRelation") - ) ++ Seq( - // SPARK-6363 Make Scala 2.11 the default Scala version - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.cleanup"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metadataCleaner"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnDriverEndpoint"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint") - ) ++ Seq( - // SPARK-7889 - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.org$apache$spark$deploy$history$HistoryServer$@tachSparkUI"), - // SPARK-13296 - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.UDFRegistration.register"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction$") - ) ++ Seq( - // SPARK-12995 Remove deprecated APIs in graphx - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.lib.SVDPlusPlus.runSVDPlusPlus"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets$default$3"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.impl.GraphImpl.mapReduceTriplets") - ) ++ Seq( - // SPARK-13426 Remove the support of SIMR - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkMasterRegex.SIMR_REGEX") - ) ++ Seq( - // SPARK-13413 Remove SparkContext.metricsSystem/schedulerBackend_ setter - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metricsSystem"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.schedulerBackend_=") - ) ++ Seq( - // SPARK-13220 Deprecate yarn-client and yarn-cluster mode - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") - ) ++ Seq( - // SPARK-13465 TaskContext. - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addTaskFailureListener") - ) ++ Seq ( - // SPARK-7729 Executor which has been killed should also be displayed on Executor Tab - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") - ) ++ Seq( - // SPARK-13526 Move SQLContext per-session states to new class - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.sql.UDFRegistration.this") - ) ++ Seq( - // [SPARK-13486][SQL] Move SQLConf into an internal package - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry$") - ) ++ Seq( - //SPARK-11011 UserDefinedType serialization should be strongly typed - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), - // SPARK-12073: backpressure rate controller consumes events preferentially from lagging partitions - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.KafkaTestUtils.createTopic"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.DirectKafkaInputDStream.maxMessagesPerPartition") - ) ++ Seq( - // [SPARK-13244][SQL] Migrates DataFrame to Dataset - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.tables"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.sql"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.baseRelationToDataFrame"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.table"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrame.apply"), - - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.LegacyFunctions"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.localSeqToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.stringRddToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.rddToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.longRddToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.intRddToDataFrameHolder"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.GroupedDataset"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.subtract"), - - // [SPARK-14451][SQL] Move encoder definition into Aggregator interface - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.toColumn"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.bufferEncoder"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.outputEncoder"), - - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MultilabelMetrics.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions") - ) ++ Seq( - // [SPARK-13686][MLLIB][STREAMING] Add a constructor parameter `reqParam` to (Streaming)LinearRegressionWithSGD - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD.this") - ) ++ Seq( - // SPARK-15250 Remove deprecated json API in DataFrameReader - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameReader.json") - ) ++ Seq( - // SPARK-13920: MIMA checks should apply to @Experimental and @DeveloperAPI APIs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineCombinersByKey"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineValuesByKey"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.run"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.runJob"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.actorSystem"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.cacheManager"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getConfigurationFromJobContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTaskAttemptIDFromTaskAttemptContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.newConfiguration"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback_="), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.canEqual"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productArity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productElement"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productIterator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productPrefix"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.setBytesReadCallback"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.updateBytesRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.canEqual"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productArity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productElement"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productIterator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productPrefix"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decFetchWaitTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decLocalBlocksFetched"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRecordsRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBlocksFetched"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBytesRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleBytesWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleWriteTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleBytesWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleWriteTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.setShuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.PCAModel.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.taskMetrics"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.TaskInfo.attempt"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.ExperimentalMethods.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUDF"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUdf"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.cumeDist"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.denseRank"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.inputFileName"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.isNaN"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.percentRank"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.rowNumber"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.sparkPartitionId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.externalBlockStoreSize"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsedByRdd"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatusListener.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.scheduler.BatchInfo.streamIdToNumRecords"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.storageStatusList"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.storage.StorageListener.storageStatusList"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.apply"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.copy"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.InputMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.OutputMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Pipeline.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PipelineModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transformImpl"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.extractLabeledPoints"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.GBTClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayes.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRest.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRestModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.RandomForestClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeans.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logLikelihood"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logPerplexity"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.RegressionEvaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Binarizer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Bucketizer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelector.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.HashingTF.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDF.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDFModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IndexToString.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Interaction.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScaler.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCA.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCAModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormula.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormulaModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.SQLTransformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScaler.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StopWordsRemover.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorAssembler.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorSlicer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2Vec.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALS.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.GBTRegressor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.extractWeightedLabeledPoints"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.extractWeightedLabeledPoints"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegression.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionTrainingSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidator.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidatorModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplit.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplitModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.BinaryClassificationMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.RegressionMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameWriter.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.broadcast"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.callUDF"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.InsertableRelation.insert"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.fMeasureByThreshold"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.pr"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.precisionByThreshold"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.predictions"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.recallByThreshold"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.roc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.describeTopics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.getVectors"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.itemFactors"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.userFactors"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.predictions"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.residuals"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.name"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.value"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.drop"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.fill"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.replace"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.jdbc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.json"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.load"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.orc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.parquet"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.table"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.text"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.crosstab"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.freqItems"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.sampleBy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.emptyDataFrame"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.range"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.functions.udf"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.JobLogger"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorHelper"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics$"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics$"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics$"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.functions$"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Estimator.fit"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Predictor.train"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Transformer.transform"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListener.onOtherEvent"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.InsertableRelation.insert") - ) ++ Seq( - // [SPARK-13926] Automatically use Kryo serializer when shuffling RDDs with simple types - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ShuffleDependency.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ShuffleDependency.serializer"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.Serializer$") - ) ++ Seq( - // SPARK-13927: add row/column iterator to local matrices - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.rowIter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.colIter") - ) ++ Seq( - // SPARK-13948: MiMa Check should catch if the visibility change to `private` - // TODO(josh): Some of these may be legitimate incompatibilities; we should follow up before the 2.0.0 release - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.toDS"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.askTimeout"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.lookupTimeout"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.UnaryTransformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.select"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.toDF"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Logging.initializeLogIfNecessary"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerEvent.logEvent"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance") - ) ++ Seq( - // [SPARK-14014] Replace existing analysis.Catalog with SessionCatalog - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.this") - ) ++ Seq( - // [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Logging"), - (problem: Problem) => problem match { - case MissingTypesProblem(_, missing) - if missing.map(_.fullName).sameElements(Seq("org.apache.spark.Logging")) => false - case _ => true - } - ) ++ Seq( - // [SPARK-13990] Automatically pick serializer when caching RDDs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.uploadBlock") - ) ++ Seq( - // [SPARK-14089][CORE][MLLIB] Remove methods that has been deprecated since 1.1, 1.2, 1.3, 1.4, and 1.5 - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.getThreadLocal"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeReduce"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeAggregate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.tree.configuration.Strategy.defaultStategy"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.saveLabeledData"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLabeledData"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.optimization.LBFGS.setMaxNumIterations"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.setScoreCol") - ) ++ Seq( - // [SPARK-14205][SQL] remove trait Queryable - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.Dataset") - ) ++ Seq( - // [SPARK-11262][ML] Unit test for gradient, loss layers, memory management - // for multilayer perceptron. - // This class is marked as `private`. - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.ann.SoftmaxFunction") - ) ++ Seq( - // [SPARK-13674][SQL] Add wholestage codegen support to Sample - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.util.random.PoissonSampler.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.random.PoissonSampler.this") - ) ++ Seq( - // [SPARK-13430][ML] moved featureCol from LinearRegressionModelSummary to LinearRegressionSummary - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this") - ) ++ Seq( - // [SPARK-14437][Core] Use the address that NettyBlockTransferService listens to create BlockManagerId - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.this") - ) ++ Seq( - // [SPARK-13048][ML][MLLIB] keepLastCheckpoint option for LDA EM optimizer - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.DistributedLDAModel.this") - ) ++ Seq( - // [SPARK-14475] Propagate user-defined context from driver to executors - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getLocalProperty"), - // [SPARK-14617] Remove deprecated APIs in TaskMetrics - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.InputMetrics$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.OutputMetrics$"), - // [SPARK-14628] Simplify task metrics by always tracking read/write metrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.readMethod"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.writeMethod") - ) ++ Seq( - // SPARK-14628: Always track input/output/shuffle metrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.totalBlocksFetched"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.inputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.outputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleWriteMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleReadMetrics"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.inputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.outputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleWriteMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleReadMetrics"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") - ) ++ Seq( - // SPARK-13643: Move functionality from SQLContext to SparkSession - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.getSchema") - ) ++ Seq( - // [SPARK-14407] Hides HadoopFsRelation related data source API into execution package - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriter"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriterFactory") - ) ++ Seq( - // SPARK-14734: Add conversions between mllib and ml Vector, Matrix types - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asML"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asML") - ) ++ Seq( - // SPARK-14704: Create accumulators in TaskMetrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.this") - ) ++ Seq( - // SPARK-14861: Replace internal usages of SQLContext with SparkSession - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.clustering.LocalLDAModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.clustering.DistributedLDAModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.clustering.LDAModel.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.ml.clustering.LDAModel.sqlContext"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.sql.Dataset.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.sql.DataFrameReader.this") - ) ++ Seq( - // SPARK-14542 configurable buffer size for pipe RDD - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.pipe"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.pipe") - ) ++ Seq( - // [SPARK-4452][Core]Shuffle data structures can starve others on the same thread for memory - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.util.collection.Spillable") - ) ++ Seq( - // [SPARK-14952][Core][ML] Remove methods deprecated in 1.6 - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.input.PortableDataStream.close"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.weights"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionModel.weights") - ) ++ Seq( - // [SPARK-10653] [Core] Remove unnecessary things from SparkEnv - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.sparkFilesDir"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.blockTransferService") - ) ++ Seq( - // SPARK-14654: New accumulator API - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ExceptionFailure$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.metrics"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.remoteBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.totalBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") - ) ++ Seq( - // [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.getOldDocConcentration"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.estimatedDocConcentration"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.topicsMatrix"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.clusterCenters"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.decodeLabel"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.encodeLabeledPoint"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.weights"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.predict"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.predictRaw"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.raw2probabilityInPlace"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.theta"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.pi"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.probability2prediction"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predictRaw"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2prediction"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2probabilityInPlace"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predict"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.coefficients"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.raw2prediction"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.getScalingVec"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.setScalingVec"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PCAModel.pc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMax"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMin"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.IDFModel.idf"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.mean"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.std"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predict"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.coefficients"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predictQuantiles"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.predictions"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.boundaries"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.predict"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.coefficients"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.this") - ) ++ Seq( - // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Private"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") - ) ++ Seq( - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asBreeze"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asBreeze") - ) ++ Seq( - // [SPARK-15914] Binary compatibility is broken since consolidation of Dataset and DataFrame - // in Spark 2.0. However, source level compatibility is still maintained. - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.load"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonFile"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") - ) ++ Seq( - // SPARK-17096: Improve exception string reported through the StreamingQueryListener - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.stackTrace"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.this") - ) ++ Seq( - // SPARK-17406 limit timeline executor events - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorIdToData"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksActive"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksComplete"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputRecords"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksFailed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleWrite"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToDuration"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputBytes"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToLogUrls"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputBytes"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputRecords"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTotalCores"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksMax"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToJvmGCTime") - ) ++ Seq( - // [SPARK-17163] Unify logistic regression interface. Private constructor has new signature. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this") - ) ++ Seq( - // [SPARK-17498] StringIndexer enhancement for handling unseen labels - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.StringIndexer"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.StringIndexerModel") - ) ++ Seq( - // [SPARK-17365][Core] Remove/Kill multiple executors together to reduce RPC call time - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.SparkContext") - ) ++ Seq( - // [SPARK-12221] Add CPU time to metrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskMetrics.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") - ) ++ Seq( - // [SPARK-18481] ML 2.1 QA: Remove deprecated methods for ML - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.PipelineStage.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.param.JavaParams.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.param.Params.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegression.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassifier.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.numTrees"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.setLabelCol"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.evaluation.Evaluator.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressor.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.validateParams"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.model"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.numTrees"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.RandomForestClassifier"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.GBTClassifier"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.GBTClassificationModel"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.RandomForestRegressor"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.GBTRegressor"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.GBTRegressionModel"), - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.getNumTrees"), - ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.getNumTrees"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.numTrees"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setFeatureSubsetStrategy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.numTrees"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy") - ) ++ Seq( - // [SPARK-21680][ML][MLLIB]optimzie Vector coompress - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.toSparseWithSize"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Vector.toSparseWithSize") - ) ++ Seq( - // [SPARK-3181][ML]Implement huber loss for LinearRegression. - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.org$apache$spark$ml$param$shared$HasLoss$_setter_$loss_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.getLoss"), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.loss") - ) - } - - def excludes(version: String) = version match { - case v if v.startsWith("3.1") => v31excludes - case v if v.startsWith("3.0") => v30excludes - case v if v.startsWith("2.4") => v24excludes - case v if v.startsWith("2.3") => v23excludes - case v if v.startsWith("2.2") => v22excludes - case v if v.startsWith("2.1") => v21excludes - case v if v.startsWith("2.0") => v20excludes - case _ => Seq() - } -} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala deleted file mode 100644 index eb12f2f1f6ab..000000000000 --- a/project/SparkBuild.scala +++ /dev/null @@ -1,1058 +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 java.io._ -import java.nio.charset.StandardCharsets.UTF_8 -import java.nio.file.Files -import java.util.Locale - -import scala.io.Source -import scala.util.Properties -import scala.collection.JavaConverters._ -import scala.collection.mutable.Stack - -import sbt._ -import sbt.Classpaths.publishTask -import sbt.Keys._ -import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion -import com.etsy.sbt.checkstyle.CheckstylePlugin.autoImport._ -import com.simplytyped.Antlr4Plugin._ -import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} -import com.typesafe.tools.mima.plugin.MimaKeys -import org.scalastyle.sbt.ScalastylePlugin.autoImport._ -import org.scalastyle.sbt.Tasks - -import spray.revolver.RevolverPlugin._ - -object BuildCommons { - - private val buildLocation = file(".").getAbsoluteFile.getParentFile - - val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, tokenProviderKafka010, sqlKafka010, avro) = Seq( - "catalyst", "sql", "hive", "hive-thriftserver", "token-provider-kafka-0-10", "sql-kafka-0-10", "avro" - ).map(ProjectRef(buildLocation, _)) - - val streamingProjects@Seq(streaming, streamingKafka010) = - Seq("streaming", "streaming-kafka-0-10").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 - - val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, - sparkGangliaLgpl, streamingKinesisAsl, - dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests) = - Seq("kubernetes", "mesos", "yarn", - "ganglia-lgpl", "streaming-kinesis-asl", - "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests").map(ProjectRef(buildLocation, _)) - - val assemblyProjects@Seq(networkYarn, streamingKafka010Assembly, streamingKinesisAslAssembly) = - Seq("network-yarn", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") - .map(ProjectRef(buildLocation, _)) - - val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples") - .map(ProjectRef(buildLocation, _)) - - val tools = ProjectRef(buildLocation, "tools") - // Root project. - val spark = ProjectRef(buildLocation, "spark") - val sparkHome = buildLocation - - val testTempDir = s"$sparkHome/target/tmp" - - val javaVersion = settingKey[String]("source and target JVM version for javac and scalac") -} - -object SparkBuild extends PomBuild { - - import BuildCommons._ - import scala.collection.mutable.Map - - val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty - - override val profiles = { - val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") - .orElse(Properties.propOrNone("sbt.maven.profiles")) match { - case None => Seq("sbt") - case Some(v) => - v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq - } - - // TODO: revisit for Scala 2.13 support - /* - Option(System.getProperty("scala.version")) - .filter(_.startsWith("2.11")) - .foreach { versionString => - System.setProperty("scala-2.11", "true") - } - if (System.getProperty("scala-2.11") == "") { - // To activate scala-2.10 profile, replace empty property value to non-empty value - // in the same way as Maven which handles -Dname as -Dname=true before executes build process. - // see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082 - System.setProperty("scala-2.11", "true") - } - */ - profiles - } - - Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { - case Some(v) => - v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => - } - - override val userPropertiesMap = System.getProperties.asScala.toMap - - lazy val MavenCompile = config("m2r") extend(Compile) - lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") - - lazy val sparkGenjavadocSettings: Seq[sbt.Def.Setting[_]] = Seq( - libraryDependencies += compilerPlugin( - "com.typesafe.genjavadoc" %% "genjavadoc-plugin" % unidocGenjavadocVersion.value cross CrossVersion.full), - scalacOptions ++= Seq( - "-P:genjavadoc:out=" + (target.value / "java"), - "-P:genjavadoc:strictVisibility=true" // hide package private types - ) - ) - - lazy val scalaStyleRules = Project("scalaStyleRules", file("scalastyle")) - .settings( - libraryDependencies += "org.scalastyle" %% "scalastyle" % "1.0.0" - ) - - lazy val scalaStyleOnCompile = taskKey[Unit]("scalaStyleOnCompile") - - lazy val scalaStyleOnTest = taskKey[Unit]("scalaStyleOnTest") - - // We special case the 'println' lint rule to only be a warning on compile, because adding - // printlns for debugging is a common use case and is easy to remember to remove. - val scalaStyleOnCompileConfig: String = { - val in = "scalastyle-config.xml" - val out = "scalastyle-on-compile.generated.xml" - val replacements = Map( - """customId="println" level="error"""" -> """customId="println" level="warn"""" - ) - var contents = Source.fromFile(in).getLines.mkString("\n") - for ((k, v) <- replacements) { - require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.") - contents = contents.replace(k, v) - } - new PrintWriter(out) { - write(contents) - close() - } - out - } - - // Return a cached scalastyle task for a given configuration (usually Compile or Test) - private def cachedScalaStyle(config: Configuration) = Def.task { - val logger = streams.value.log - // We need a different cache dir per Configuration, otherwise they collide - val cacheDir = target.value / s"scalastyle-cache-${config.name}" - val cachedFun = FileFunction.cached(cacheDir, FilesInfo.lastModified, FilesInfo.exists) { - (inFiles: Set[File]) => { - val args: Seq[String] = Seq.empty - val scalaSourceV = Seq(file(scalaSource.in(config).value.getAbsolutePath)) - val configV = (baseDirectory in ThisBuild).value / scalaStyleOnCompileConfig - val configUrlV = scalastyleConfigUrl.in(config).value - val streamsV = streams.in(config).value - val failOnErrorV = true - val failOnWarningV = false - val scalastyleTargetV = scalastyleTarget.in(config).value - val configRefreshHoursV = scalastyleConfigRefreshHours.in(config).value - val targetV = target.in(config).value - val configCacheFileV = scalastyleConfigUrlCacheFile.in(config).value - - logger.info(s"Running scalastyle on ${name.value} in ${config.name}") - Tasks.doScalastyle(args, configV, configUrlV, failOnErrorV, failOnWarningV, scalaSourceV, - scalastyleTargetV, streamsV, configRefreshHoursV, targetV, configCacheFileV) - - Set.empty - } - } - - cachedFun(findFiles(scalaSource.in(config).value)) - } - - private def findFiles(file: File): Set[File] = if (file.isDirectory) { - file.listFiles().toSet.flatMap(findFiles) + file - } else { - Set(file) - } - - def enableScalaStyle: Seq[sbt.Def.Setting[_]] = Seq( - scalaStyleOnCompile := cachedScalaStyle(Compile).value, - scalaStyleOnTest := cachedScalaStyle(Test).value, - logLevel in scalaStyleOnCompile := Level.Warn, - logLevel in scalaStyleOnTest := Level.Warn, - (compile in Compile) := { - scalaStyleOnCompile.value - (compile in Compile).value - }, - (compile in Test) := { - scalaStyleOnTest.value - (compile in Test).value - } - ) - - lazy val sharedSettings = sparkGenjavadocSettings ++ - (if (sys.env.contains("NOLINT_ON_COMPILE")) Nil else enableScalaStyle) ++ Seq( - exportJars in Compile := true, - exportJars in Test := false, - javaHome := sys.env.get("JAVA_HOME") - .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) - .map(file), - incOptions := incOptions.value.withNameHashing(true), - publishMavenStyle := true, - unidocGenjavadocVersion := "0.16", - - // 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/maven2/", - DefaultMavenRepository, - Resolver.mavenLocal, - Resolver.file("local", file(Path.userHome.absolutePath + "/.ivy2/local"))(Resolver.ivyStylePatterns) - ), - externalResolvers := resolvers.value, - otherResolvers := SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))).value, - publishLocalConfiguration in MavenCompile := - new PublishConfiguration(None, "dotM2", packagedArtifacts.value, Seq(), ivyLoggingLevel.value), - publishMavenStyle in MavenCompile := true, - publishLocal in MavenCompile := publishTask(publishLocalConfiguration in MavenCompile, deliverLocal).value, - publishLocalBoth := Seq(publishLocal in MavenCompile, publishLocal).dependOn.value, - - javacOptions in (Compile, doc) ++= { - val versionParts = System.getProperty("java.version").split("[+.\\-]+", 3) - var major = versionParts(0).toInt - if (major == 1) major = versionParts(1).toInt - if (major >= 8) Seq("-Xdoclint:all", "-Xdoclint:-missing") else Seq.empty - }, - - javaVersion := SbtPomKeys.effectivePom.value.getProperties.get("java.version").asInstanceOf[String], - - javacOptions in Compile ++= Seq( - "-encoding", UTF_8.name(), - "-source", javaVersion.value - ), - // This -target and Xlint:unchecked options cannot be set in the Compile configuration scope since - // `javadoc` doesn't play nicely with them; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629 - // for additional discussion and explanation. - javacOptions in (Compile, compile) ++= Seq( - "-target", javaVersion.value, - "-Xlint:unchecked" - ), - - scalacOptions in Compile ++= Seq( - s"-target:jvm-${javaVersion.value}", - "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath // Required for relative source links in scaladoc - ), - - // Remove certain packages from Scaladoc - scalacOptions in (Compile, doc) := Seq( - "-groups", - "-skip-packages", Seq( - "org.apache.spark.api.python", - "org.apache.spark.network", - "org.apache.spark.deploy", - "org.apache.spark.util.collection" - ).mkString(":"), - "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc" - ) ++ { - // Do not attempt to scaladoc javadoc comments under 2.12 since it can't handle inner classes - if (scalaBinaryVersion.value == "2.12") Seq("-no-java-comments") else Seq.empty - }, - - // Implements -Xfatal-warnings, ignoring deprecation warnings. - // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410. - compile in Compile := { - val analysis = (compile in Compile).value - val out = streams.value - - def logProblem(l: (=> String) => Unit, f: File, p: xsbti.Problem) = { - l(f.toString + ":" + p.position.line.fold("")(_ + ":") + " " + p.message) - l(p.position.lineContent) - l("") - } - - var failed = 0 - analysis.infos.allInfos.foreach { case (k, i) => - i.reportedProblems foreach { p => - val deprecation = p.message.contains("is deprecated") - - if (!deprecation) { - failed = failed + 1 - } - - val printer: (=> String) => Unit = s => if (deprecation) { - out.log.warn(s) - } else { - out.log.error("[warn] " + s) - } - - logProblem(printer, k, p) - - } - } - - if (failed > 0) { - sys.error(s"$failed fatal warnings") - } - analysis - } - ) - - def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = { - val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]()) - projectsMap += (projectRef.project -> (existingSettings ++ settings)) - } - - // Note ordering of these settings matter. - /* Enable shared settings on all projects */ - (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ copyJarsProjects ++ Seq(spark, tools)) - .foreach(enable(sharedSettings ++ DependencyOverrides.settings ++ - ExcludedDependencies.settings ++ Checkstyle.settings)) - - /* Enable tests settings for all projects except examples, assembly and tools */ - (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) - - val mimaProjects = allProjects.filterNot { x => - Seq( - spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro - ).contains(x) - } - - mimaProjects.foreach { x => - enable(MimaBuild.mimaSettings(sparkHome, x))(x) - } - - /* Generate and pick the spark build info from extra-resources */ - enable(Core.settings)(core) - - /* Unsafe settings */ - enable(Unsafe.settings)(unsafe) - - /* - * Set up tasks to copy dependencies during packaging. This step can be disabled in the command - * line, so that dev/mima can run without trying to copy these files again and potentially - * causing issues. - */ - if (!"false".equals(System.getProperty("copyDependencies"))) { - copyJarsProjects.foreach(enable(CopyDependencies.settings)) - } - - /* Enable Assembly for all assembly projects */ - assemblyProjects.foreach(enable(Assembly.settings)) - - /* Package pyspark artifacts in a separate zip file for YARN. */ - enable(PySparkAssembly.settings)(assembly) - - /* Enable unidoc only for the root spark project */ - enable(Unidoc.settings)(spark) - - /* Catalyst ANTLR generation settings */ - enable(Catalyst.settings)(catalyst) - - /* Spark SQL Core console settings */ - enable(SQL.settings)(sql) - - /* Hive console settings */ - enable(Hive.settings)(hive) - - // SPARK-14738 - Remove docker tests from main Spark build - // enable(DockerIntegrationTests.settings)(dockerIntegrationTests) - - enable(KubernetesIntegrationTests.settings)(kubernetesIntegrationTests) - - /** - * Adds the ability to run the spark shell directly from SBT without building an assembly - * jar. - * - * Usage: `build/sbt sparkShell` - */ - val sparkShell = taskKey[Unit]("start a spark-shell.") - val sparkPackage = inputKey[Unit]( - s""" - |Download and run a spark package. - |Usage `builds/sbt "sparkPackage [args] - """.stripMargin) - val sparkSql = taskKey[Unit]("starts the spark sql CLI.") - - enable(Seq( - connectInput in run := true, - fork := true, - outputStrategy in run := Some (StdoutOutput), - - javaOptions += "-Xmx2g", - - sparkShell := { - (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value - }, - - sparkPackage := { - import complete.DefaultParsers._ - val packages :: className :: otherArgs = spaceDelimited(" [args]").parsed.toList - val scalaRun = (runner in run).value - val classpath = (fullClasspath in Runtime).value - val args = Seq("--packages", packages, "--class", className, (Keys.`package` in Compile in LocalProject("core")) - .value.getCanonicalPath) ++ otherArgs - println(args) - scalaRun.run("org.apache.spark.deploy.SparkSubmit", classpath.map(_.data), args, streams.value.log) - }, - - javaOptions in Compile += "-Dspark.master=local", - - sparkSql := { - (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value - } - ))(assembly) - - enable(Seq(sparkShell := sparkShell in LocalProject("assembly")))(spark) - - // TODO: move this to its upstream project. - override def projectDefinitions(baseDirectory: File): Seq[Project] = { - super.projectDefinitions(baseDirectory).map { x => - if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*) - else x.settings(Seq[Setting[_]](): _*) - } ++ Seq[Project](OldDeps.project) - } - - if (!sys.env.contains("SERIAL_SBT_TESTS")) { - allProjects.foreach(enable(SparkParallelTestGrouping.settings)) - } -} - -object SparkParallelTestGrouping { - // Settings for parallelizing tests. The basic strategy here is to run the slowest suites (or - // collections of suites) in their own forked JVMs, allowing us to gain parallelism within a - // SBT project. Here, we take a whitelisting approach where the default behavior is to run all - // tests sequentially in a single JVM, requiring us to manually opt-in to the extra parallelism. - // - // There are a reasons why such a whitelist approach is good: - // - // 1. Launching one JVM per suite adds significant overhead for short-running suites. In - // addition to JVM startup time and JIT warmup, it appears that initialization of Derby - // metastores can be very slow so creating a fresh warehouse per suite is inefficient. - // - // 2. When parallelizing within a project we need to give each forked JVM a different tmpdir - // so that the metastore warehouses do not collide. Unfortunately, it seems that there are - // some tests which have an overly tight dependency on the default tmpdir, so those fragile - // tests need to continue re-running in the default configuration (or need to be rewritten). - // Fixing that problem would be a huge amount of work for limited payoff in most cases - // because most test suites are short-running. - // - - private val testsWhichShouldRunInTheirOwnDedicatedJvm = Set( - "org.apache.spark.DistributedSuite", - "org.apache.spark.sql.catalyst.expressions.DateExpressionsSuite", - "org.apache.spark.sql.catalyst.expressions.HashExpressionsSuite", - "org.apache.spark.sql.catalyst.expressions.CastSuite", - "org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite", - "org.apache.spark.sql.hive.HiveExternalCatalogSuite", - "org.apache.spark.sql.hive.StatisticsSuite", - "org.apache.spark.sql.hive.execution.HiveCompatibilitySuite", - "org.apache.spark.sql.hive.client.VersionsSuite", - "org.apache.spark.sql.hive.client.HiveClientVersions", - "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite", - "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.SparkSQLEnvSuite", - "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", - "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", - "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" - ) - - private val DEFAULT_TEST_GROUP = "default_test_group" - - private def testNameToTestGroup(name: String): String = name match { - case _ if testsWhichShouldRunInTheirOwnDedicatedJvm.contains(name) => name - case _ => DEFAULT_TEST_GROUP - } - - lazy val settings = Seq( - testGrouping in Test := { - val tests: Seq[TestDefinition] = (definedTests in Test).value - val defaultForkOptions = ForkOptions( - bootJars = Nil, - javaHome = javaHome.value, - connectInput = connectInput.value, - outputStrategy = outputStrategy.value, - runJVMOptions = (javaOptions in Test).value, - workingDirectory = Some(baseDirectory.value), - envVars = (envVars in Test).value - ) - tests.groupBy(test => testNameToTestGroup(test.name)).map { case (groupName, groupTests) => - val forkOptions = { - if (groupName == DEFAULT_TEST_GROUP) { - defaultForkOptions - } else { - defaultForkOptions.copy(runJVMOptions = defaultForkOptions.runJVMOptions ++ - Seq(s"-Djava.io.tmpdir=${baseDirectory.value}/target/tmp/$groupName")) - } - } - new Tests.Group( - name = groupName, - tests = groupTests, - runPolicy = Tests.SubProcess(forkOptions)) - } - }.toSeq - ) -} - -object Core { - lazy val settings = Seq( - resourceGenerators in Compile += Def.task { - val buildScript = baseDirectory.value + "/../build/spark-build-info" - val targetDir = baseDirectory.value + "/target/extra-resources/" - val command = Seq("bash", buildScript, targetDir, version.value) - Process(command).!! - val propsFile = baseDirectory.value / "target" / "extra-resources" / "spark-version-info.properties" - Seq(propsFile) - }.taskValue - ) -} - -object Unsafe { - lazy val settings = Seq( - // This option is needed to suppress warnings from sun.misc.Unsafe usage - javacOptions in Compile += "-XDignore.symbol.file" - ) -} - - -object DockerIntegrationTests { - // This serves to override the override specified in DependencyOverrides: - lazy val settings = Seq( - dependencyOverrides += "com.google.guava" % "guava" % "18.0", - resolvers += "DB2" at "https://app.camunda.com/nexus/content/repositories/public/", - libraryDependencies += "com.oracle" % "ojdbc6" % "11.2.0.1.0" from "https://app.camunda.com/nexus/content/repositories/public/com/oracle/ojdbc6/11.2.0.1.0/ojdbc6-11.2.0.1.0.jar" // scalastyle:ignore - ) -} - -/** - * These settings run a hardcoded configuration of the Kubernetes integration tests using - * minikube. Docker images will have the "dev" tag, and will be overwritten every time the - * integration tests are run. The integration tests are actually bound to the "test" phase, - * so running "test" on this module will run the integration tests. - * - * There are two ways to run the tests: - * - the "tests" task builds docker images and runs the test, so it's a little slow. - * - the "run-its" task just runs the tests on a pre-built set of images. - * - * Note that this does not use the shell scripts that the maven build uses, which are more - * configurable. This is meant as a quick way for developers to run these tests against their - * local changes. - */ -object KubernetesIntegrationTests { - import BuildCommons._ - - val dockerBuild = TaskKey[Unit]("docker-imgs", "Build the docker images for ITs.") - val runITs = TaskKey[Unit]("run-its", "Only run ITs, skip image build.") - val imageTag = settingKey[String]("Tag to use for images built during the test.") - val namespace = settingKey[String]("Namespace where to run pods.") - - // Hack: this variable is used to control whether to build docker images. It's updated by - // the tasks below in a non-obvious way, so that you get the functionality described in - // the scaladoc above. - private var shouldBuildImage = true - - lazy val settings = Seq( - imageTag := "dev", - namespace := "default", - dockerBuild := { - if (shouldBuildImage) { - val dockerTool = s"$sparkHome/bin/docker-image-tool.sh" - val bindingsDir = s"$sparkHome/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings" - val cmd = Seq(dockerTool, "-m", - "-t", imageTag.value, - "-p", s"$bindingsDir/python/Dockerfile", - "-R", s"$bindingsDir/R/Dockerfile", - "build" - ) - val ec = Process(cmd).! - if (ec != 0) { - throw new IllegalStateException(s"Process '${cmd.mkString(" ")}' exited with $ec.") - } - } - shouldBuildImage = true - }, - runITs := Def.taskDyn { - shouldBuildImage = false - Def.task { - (test in Test).value - } - }.value, - test in Test := (test in Test).dependsOn(dockerBuild).value, - javaOptions in Test ++= Seq( - "-Dspark.kubernetes.test.deployMode=minikube", - s"-Dspark.kubernetes.test.imageTag=${imageTag.value}", - s"-Dspark.kubernetes.test.namespace=${namespace.value}", - s"-Dspark.kubernetes.test.unpackSparkDir=$sparkHome" - ), - // Force packaging before building images, so that the latest code is tested. - dockerBuild := dockerBuild.dependsOn(packageBin in Compile in assembly) - .dependsOn(packageBin in Compile in examples).value - ) -} - -/** - * Overrides to work around sbt's dependency resolution being different from Maven's. - */ -object DependencyOverrides { - lazy val settings = Seq( - dependencyOverrides += "com.google.guava" % "guava" % "14.0.1", - dependencyOverrides += "xerces" % "xercesImpl" % "2.12.0", - dependencyOverrides += "jline" % "jline" % "2.14.6", - dependencyOverrides += "org.apache.avro" % "avro" % "1.8.2") -} - -/** - * This excludes library dependencies in sbt, which are specified in maven but are - * not needed by sbt build. - */ -object ExcludedDependencies { - lazy val settings = Seq( - libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") } - ) -} - -/** - * Project to pull previous artifacts of Spark for generating Mima excludes. - */ -object OldDeps { - - lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings) - - lazy val allPreviousArtifactKeys = Def.settingDyn[Seq[Set[ModuleID]]] { - SparkBuild.mimaProjects - .map { project => MimaKeys.mimaPreviousArtifacts in project } - .map(k => Def.setting(k.value)) - .join - } - - def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq( - name := "old-deps", - libraryDependencies := allPreviousArtifactKeys.value.flatten - ) -} - -object Catalyst { - lazy val settings = antlr4Settings ++ Seq( - antlr4Version in Antlr4 := SbtPomKeys.effectivePom.value.getProperties.get("antlr4.version").asInstanceOf[String], - antlr4PackageName in Antlr4 := Some("org.apache.spark.sql.catalyst.parser"), - antlr4GenListener in Antlr4 := true, - antlr4GenVisitor in Antlr4 := true, - antlr4TreatWarningsAsErrors in Antlr4 := true - ) -} - -object SQL { - lazy val settings = Seq( - initialCommands in console := - """ - |import org.apache.spark.SparkContext - |import org.apache.spark.sql.SQLContext - |import org.apache.spark.sql.catalyst.analysis._ - |import org.apache.spark.sql.catalyst.dsl._ - |import org.apache.spark.sql.catalyst.errors._ - |import org.apache.spark.sql.catalyst.expressions._ - |import org.apache.spark.sql.catalyst.plans.logical._ - |import org.apache.spark.sql.catalyst.rules._ - |import org.apache.spark.sql.catalyst.util._ - |import org.apache.spark.sql.execution - |import org.apache.spark.sql.functions._ - |import org.apache.spark.sql.types._ - | - |val sc = new SparkContext("local[*]", "dev-shell") - |val sqlContext = new SQLContext(sc) - |import sqlContext.implicits._ - |import sqlContext._ - """.stripMargin, - cleanupCommands in console := "sc.stop()" - ) -} - -object Hive { - - lazy val settings = Seq( - // Specially disable assertions since some Hive tests fail them - javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"), - // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings - // only for this subproject. - scalacOptions := (scalacOptions map { currentOpts: Seq[String] => - currentOpts.filterNot(_ == "-deprecation") - }).value, - initialCommands in console := - """ - |import org.apache.spark.SparkContext - |import org.apache.spark.sql.catalyst.analysis._ - |import org.apache.spark.sql.catalyst.dsl._ - |import org.apache.spark.sql.catalyst.errors._ - |import org.apache.spark.sql.catalyst.expressions._ - |import org.apache.spark.sql.catalyst.plans.logical._ - |import org.apache.spark.sql.catalyst.rules._ - |import org.apache.spark.sql.catalyst.util._ - |import org.apache.spark.sql.execution - |import org.apache.spark.sql.functions._ - |import org.apache.spark.sql.hive._ - |import org.apache.spark.sql.hive.test.TestHive._ - |import org.apache.spark.sql.hive.test.TestHive.implicits._ - |import org.apache.spark.sql.types._""".stripMargin, - cleanupCommands in console := "sparkContext.stop()", - // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce - // in order to generate golden files. This is only required for developers who are adding new - // new query tests. - fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") } - ) -} - -object Assembly { - import sbtassembly.AssemblyUtils._ - import sbtassembly.Plugin._ - import AssemblyKeys._ - - val hadoopVersion = taskKey[String]("The version of hadoop that spark is compiled against.") - - lazy val settings = assemblySettings ++ Seq( - test in assembly := {}, - hadoopVersion := { - sys.props.get("hadoop.version") - .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) - }, - jarName in assembly := { - if (moduleName.value.contains("streaming-kafka-0-10-assembly") - || moduleName.value.contains("streaming-kinesis-asl-assembly")) { - s"${moduleName.value}-${version.value}.jar" - } else { - s"${moduleName.value}-${version.value}-hadoop${hadoopVersion.value}.jar" - } - }, - jarName in (Test, assembly) := s"${moduleName.value}-test-${version.value}.jar", - mergeStrategy in assembly := { - case m if m.toLowerCase(Locale.ROOT).endsWith("manifest.mf") - => MergeStrategy.discard - case m if m.toLowerCase(Locale.ROOT).matches("meta-inf.*\\.sf$") - => MergeStrategy.discard - case "log4j.properties" => MergeStrategy.discard - case m if m.toLowerCase(Locale.ROOT).startsWith("meta-inf/services/") - => MergeStrategy.filterDistinctLines - case "reference.conf" => MergeStrategy.concat - case _ => MergeStrategy.first - } - ) -} - -object PySparkAssembly { - import sbtassembly.Plugin._ - import AssemblyKeys._ - import java.util.zip.{ZipOutputStream, ZipEntry} - - lazy val settings = Seq( - // Use a resource generator to copy all .py files from python/pyspark into a managed directory - // to be included in the assembly. We can't just add "python/" to the assembly's resource dir - // list since that will copy unneeded / unwanted files. - resourceGenerators in Compile += Def.macroValueI(resourceManaged in Compile map { outDir: File => - val src = new File(BuildCommons.sparkHome, "python/pyspark") - val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip") - zipFile.delete() - zipRecursive(src, zipFile) - Seq.empty[File] - }).value - ) - - private def zipRecursive(source: File, destZipFile: File) = { - val destOutput = new ZipOutputStream(new FileOutputStream(destZipFile)) - addFilesToZipStream("", source, destOutput) - destOutput.flush() - destOutput.close() - } - - private def addFilesToZipStream(parent: String, source: File, output: ZipOutputStream): Unit = { - if (source.isDirectory()) { - output.putNextEntry(new ZipEntry(parent + source.getName())) - for (file <- source.listFiles()) { - addFilesToZipStream(parent + source.getName() + File.separator, file, output) - } - } else { - val in = new FileInputStream(source) - output.putNextEntry(new ZipEntry(parent + source.getName())) - val buf = new Array[Byte](8192) - var n = 0 - while (n != -1) { - n = in.read(buf) - if (n != -1) { - output.write(buf, 0, n) - } - } - output.closeEntry() - in.close() - } - } - -} - -object Unidoc { - - import BuildCommons._ - import sbtunidoc.Plugin._ - import UnidocKeys._ - - private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = { - packages - .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(_.getCanonicalPath.contains("org/apache/spark/rpc"))) - .map(_.filterNot(f => - 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(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"))) - .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) - .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution"))) - .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] = { - classpaths - .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka-clients-0\.10.*"""))) - .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka_2\..*-0\.10.*"""))) - } - - val unidocSourceBase = settingKey[String]("Base URL of source links in Scaladoc.") - - lazy val settings = scalaJavaUnidocSettings ++ Seq ( - publish := {}, - - unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, - yarn, tags, streamingKafka010, sqlKafka010, avro), - unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, - yarn, tags, streamingKafka010, sqlKafka010, avro), - - unidocAllClasspaths in (ScalaUnidoc, unidoc) := { - ignoreClasspaths((unidocAllClasspaths in (ScalaUnidoc, unidoc)).value) - }, - - unidocAllClasspaths in (JavaUnidoc, unidoc) := { - ignoreClasspaths((unidocAllClasspaths in (JavaUnidoc, unidoc)).value) - }, - - // Skip actual catalyst, but include the subproject. - // Catalyst is not public API and contains quasiquotes which break scaladoc. - unidocAllSources in (ScalaUnidoc, unidoc) := { - ignoreUndocumentedPackages((unidocAllSources in (ScalaUnidoc, unidoc)).value) - }, - - // Skip class names containing $ and some internal packages in Javadocs - unidocAllSources in (JavaUnidoc, unidoc) := { - ignoreUndocumentedPackages((unidocAllSources in (JavaUnidoc, unidoc)).value) - .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop"))) - }, - - javacOptions in (JavaUnidoc, unidoc) := Seq( - "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", - "-public", - "-noqualifier", "java.lang", - "-tag", """example:a:Example\:""", - "-tag", """note:a:Note\:""", - "-tag", "group:X", - "-tag", "tparam:X", - "-tag", "constructor:X", - "-tag", "todo:X", - "-tag", "groupname:X" - ), - - // Use GitHub repository for Scaladoc source links - unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}", - - scalacOptions in (ScalaUnidoc, unidoc) ++= Seq( - "-groups", // Group similar methods together based on the @group annotation. - "-skip-packages", "org.apache.hadoop", - "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath - ) ++ ( - // Add links to sources when generating Scaladoc for a non-snapshot release - if (!isSnapshot.value) { - Opts.doc.sourceUrl(unidocSourceBase.value + "€{FILE_PATH}.scala") - } else { - Seq() - } - ) - ) -} - -object Checkstyle { - lazy val settings = Seq( - checkstyleSeverityLevel := Some(CheckstyleSeverityLevel.Error), - javaSource in (Compile, checkstyle) := baseDirectory.value / "src/main/java", - javaSource in (Test, checkstyle) := baseDirectory.value / "src/test/java", - checkstyleConfigLocation := CheckstyleConfigLocation.File("dev/checkstyle.xml"), - checkstyleOutputFile := baseDirectory.value / "target/checkstyle-output.xml", - checkstyleOutputFile in Test := baseDirectory.value / "target/checkstyle-output.xml" - ) -} - -object CopyDependencies { - - val copyDeps = TaskKey[Unit]("copyDeps", "Copies needed dependencies to the build directory.") - val destPath = (crossTarget in Compile) { _ / "jars"} - - lazy val settings = Seq( - copyDeps := { - val dest = destPath.value - if (!dest.isDirectory() && !dest.mkdirs()) { - throw new IOException("Failed to create jars directory.") - } - - (dependencyClasspath in Compile).value.map(_.data) - .filter { jar => jar.isFile() } - .foreach { jar => - val destJar = new File(dest, jar.getName()) - if (destJar.isFile()) { - destJar.delete() - } - Files.copy(jar.toPath(), destJar.toPath()) - } - }, - crossTarget in (Compile, packageBin) := destPath.value, - packageBin in Compile := (packageBin in Compile).dependsOn(copyDeps).value - ) - -} - -object TestSettings { - import BuildCommons._ - - // TODO revisit for Scala 2.13 support - private val scalaBinaryVersion = "2.12" - /* - if (System.getProperty("scala-2.11") == "true") { - "2.11" - } else { - "2.12" - } - */ - - private val defaultExcludedTags = Seq("org.apache.spark.tags.ChromeUITest") - - lazy val settings = Seq ( - // Fork new JVMs for tests and set Java options for those - fork := true, - // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes - // launched by the tests have access to the correct test-time classpath. - envVars in Test ++= Map( - "SPARK_DIST_CLASSPATH" -> - (fullClasspath in Test).value.files.map(_.getAbsolutePath) - .mkString(File.pathSeparator).stripSuffix(File.pathSeparator), - "SPARK_PREPEND_CLASSES" -> "1", - "SPARK_SCALA_VERSION" -> scalaBinaryVersion, - "SPARK_TESTING" -> "1", - "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), - javaOptions in Test += s"-Djava.io.tmpdir=$testTempDir", - javaOptions in Test += "-Dspark.test.home=" + sparkHome, - javaOptions in Test += "-Dspark.testing=1", - javaOptions in Test += "-Dspark.port.maxRetries=100", - javaOptions in Test += "-Dspark.master.rest.enabled=false", - javaOptions in Test += "-Dspark.memory.debugFill=true", - javaOptions in Test += "-Dspark.ui.enabled=false", - javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", - javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", - javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=false", - javaOptions in Test += "-Dderby.system.durability=test", - javaOptions in Test += "-Dio.netty.tryReflectionSetAccessible=true", - javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) - .map { case (k,v) => s"-D$k=$v" }.toSeq, - javaOptions in Test += "-ea", - // SPARK-29282 This is for consistency between JDK8 and JDK11. - javaOptions in Test ++= "-Xmx4g -Xss4m -XX:+UseParallelGC -XX:-UseDynamicNumberOfGCThreads" - .split(" ").toSeq, - javaOptions += "-Xmx3g", - // Exclude tags defined in a system property - testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, - sys.props.get("test.exclude.tags").map { tags => - tags.split(",").flatMap { tag => Seq("-l", tag) }.toSeq - }.getOrElse(Nil): _*), - testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, - sys.props.get("test.default.exclude.tags").map(tags => tags.split(",").toSeq) - .map(tags => tags.filter(!_.trim.isEmpty)).getOrElse(defaultExcludedTags) - .flatMap(tag => Seq("-l", tag)): _*), - testOptions in Test += Tests.Argument(TestFrameworks.JUnit, - sys.props.get("test.exclude.tags").map { tags => - Seq("--exclude-categories=" + tags) - }.getOrElse(Nil): _*), - // Show full stack trace and duration in test cases. - testOptions in Test += Tests.Argument("-oDF"), - testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), - // Enable Junit testing. - libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test", - // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project - // can run in parallel with one another. It does NOT control whether tests execute in parallel - // within the same JVM (which is controlled by `testForkedParallel`) or whether test cases - // within the same suite can run in parallel (which is a ScalaTest runner option which is passed - // to the underlying runner but is not a SBT-level configuration). This needs to be `true` in - // order for the extra parallelism enabled by `SparkParallelTestGrouping` to take effect. - // The `SERIAL_SBT_TESTS` check is here so the extra parallelism can be feature-flagged. - parallelExecution in Test := { if (sys.env.contains("SERIAL_SBT_TESTS")) false else true }, - // Make sure the test temp directory exists. - resourceGenerators in Test += Def.macroValueI(resourceManaged in Test map { outDir: File => - var dir = new File(testTempDir) - if (!dir.isDirectory()) { - // Because File.mkdirs() can fail if multiple callers are trying to create the same - // parent directory, this code tries to create parents one at a time, and avoids - // failures when the directories have been created by somebody else. - val stack = new Stack[File]() - while (!dir.isDirectory()) { - stack.push(dir) - dir = dir.getParentFile() - } - - while (stack.nonEmpty) { - val d = stack.pop() - require(d.mkdir() || d.isDirectory(), s"Failed to create directory $d") - } - } - Seq.empty[File] - }).value, - concurrentRestrictions in Global := { - // The number of concurrent test groups is empirically chosen based on experience - // with Jenkins flakiness. - if (sys.env.contains("SERIAL_SBT_TESTS")) (concurrentRestrictions in Global).value - else Seq(Tags.limit(Tags.ForkedTestGroup, 4)) - } - ) - -} diff --git a/project/build.properties b/project/build.properties deleted file mode 100644 index 23aa187fb35a..000000000000 --- a/project/build.properties +++ /dev/null @@ -1,17 +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. -# -sbt.version=0.13.18 diff --git a/project/plugins.sbt b/project/plugins.sbt deleted file mode 100644 index 5f21d8126e48..000000000000 --- a/project/plugins.sbt +++ /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. - */ - -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.25" - -// checkstyle uses guava 23.0. -libraryDependencies += "com.google.guava" % "guava" % "23.0" - -// need to make changes to uptake sbt 1.0 support in "com.eed3si9n" % "sbt-assembly" % "1.14.5" -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") - -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "5.2.4") - -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 -addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") - -// need to make changes to uptake sbt 1.0 support in "com.eed3si9n" % "sbt-unidoc" % "0.4.1" -addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.3") - -// need to make changes to uptake sbt 1.0 support in "com.cavorite" % "sbt-avro-1-7" % "1.1.2" -addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") - -addSbtPlugin("io.spray" % "sbt-revolver" % "0.9.1") - -libraryDependencies += "org.ow2.asm" % "asm" % "7.2" - -libraryDependencies += "org.ow2.asm" % "asm-commons" % "7.2" - -// sbt 1.0.0 support: https://github.com/ihji/sbt-antlr4/issues/14 -addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.13") - -// Spark uses a custom fork of the sbt-pom-reader plugin which contains a patch to fix issues -// related to test-jar dependencies (https://github.com/sbt/sbt-pom-reader/pull/14). The source for -// this fork is published at https://github.com/JoshRosen/sbt-pom-reader/tree/v1.0.0-spark -// and corresponds to commit b160317fcb0b9d1009635a7c5aa05d0f3be61936 in that repository. -// In the long run, we should try to merge our patch upstream and switch to an upstream version of -// the plugin; this is tracked at SPARK-14401. - -addSbtPlugin("org.spark-project" % "sbt-pom-reader" % "1.0.0-spark") 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 d6ba756f29f2..db1f6fbd97d9 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 @@ -39,8 +39,6 @@ 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 originalLegacyAllowCastNumericToTimestamp = - TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -60,7 +58,6 @@ 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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) RuleExecutor.resetMetrics() } @@ -71,8 +68,7 @@ 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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, - originalLegacyAllowCastNumericToTimestamp) + // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) } finally { diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..573541ac9702 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b b/sql/hive/src/test/resources/golden/decimal_1-10-e4e90927ac59f5920de3dc61c3288dde similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b rename to sql/hive/src/test/resources/golden/decimal_1-10-e4e90927ac59f5920de3dc61c3288dde diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 b/sql/hive/src/test/resources/golden/decimal_1-11-c20dea9d716bef1bdbdef71323b1cc5b similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 rename to sql/hive/src/test/resources/golden/decimal_1-11-c20dea9d716bef1bdbdef71323b1cc5b diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba b/sql/hive/src/test/resources/golden/decimal_1-12-f2f975b73220512d4bf2b9bd93354aba similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba rename to sql/hive/src/test/resources/golden/decimal_1-12-f2f975b73220512d4bf2b9bd93354aba diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 b/sql/hive/src/test/resources/golden/decimal_1-13-2c2325880ea79c8e308398d46c8565f8 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 rename to sql/hive/src/test/resources/golden/decimal_1-13-2c2325880ea79c8e308398d46c8565f8 diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 b/sql/hive/src/test/resources/golden/decimal_1-14-c4c33bdb9f3c6cad77552f0f353092d3 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 rename to sql/hive/src/test/resources/golden/decimal_1-14-c4c33bdb9f3c6cad77552f0f353092d3 diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 b/sql/hive/src/test/resources/golden/decimal_1-15-e45935cfffb9045394e804d0d1fc52f0 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 rename to sql/hive/src/test/resources/golden/decimal_1-15-e45935cfffb9045394e804d0d1fc52f0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 b/sql/hive/src/test/resources/golden/decimal_1-16-31ecaab3afa056fcc656d6e54f845cf4 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 rename to sql/hive/src/test/resources/golden/decimal_1-16-31ecaab3afa056fcc656d6e54f845cf4 diff --git a/sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/decimal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 rename to sql/hive/src/test/resources/golden/decimal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972 diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 b/sql/hive/src/test/resources/golden/decimal_1-3-ee665100ca5de3a006df43e97cfa707 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 rename to sql/hive/src/test/resources/golden/decimal_1-3-ee665100ca5de3a006df43e97cfa707 diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/decimal_1-4-80fc87cab17ceffea334afbb230a6653 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 rename to sql/hive/src/test/resources/golden/decimal_1-4-80fc87cab17ceffea334afbb230a6653 diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 b/sql/hive/src/test/resources/golden/decimal_1-5-5dd925bba25f735bfd6442a841afe119 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 rename to sql/hive/src/test/resources/golden/decimal_1-5-5dd925bba25f735bfd6442a841afe119 diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 b/sql/hive/src/test/resources/golden/decimal_1-6-bfab296ca5693e647e33899dfeeb256 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 rename to sql/hive/src/test/resources/golden/decimal_1-6-bfab296ca5693e647e33899dfeeb256 diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 b/sql/hive/src/test/resources/golden/decimal_1-7-a402201ed5159941384d40e09dc367a5 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 rename to sql/hive/src/test/resources/golden/decimal_1-7-a402201ed5159941384d40e09dc367a5 diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 b/sql/hive/src/test/resources/golden/decimal_1-8-2cfd7d00bc37a8e433ad005896173c1 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 rename to sql/hive/src/test/resources/golden/decimal_1-8-2cfd7d00bc37a8e433ad005896173c1 diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde b/sql/hive/src/test/resources/golden/decimal_1-9-84cd75e494d113a48c4145298177d6d8 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde rename to sql/hive/src/test/resources/golden/decimal_1-9-84cd75e494d113a48c4145298177d6d8 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 2e4c01830432..2f0f88260641 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 @@ -315,6 +315,7 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + print(cachedAnswerName) new File(answerCache, cachedAnswerName) } From f84caa10a8bbad668c95ce717c475e6931bea49c Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 12 Jun 2020 11:57:54 +0800 Subject: [PATCH 59/75] fix error cases --- project/MimaBuild.scala | 99 +++ project/MimaExcludes.scala | 1698 ++++++++++++++++++++++++++++++++++++ project/SparkBuild.scala | 1058 ++++++++++++++++++++++ project/build.properties | 17 + project/plugins.sbt | 65 ++ 5 files changed, 2937 insertions(+) create mode 100644 project/MimaBuild.scala create mode 100644 project/MimaExcludes.scala create mode 100644 project/SparkBuild.scala create mode 100644 project/build.properties create mode 100644 project/plugins.sbt diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala new file mode 100644 index 000000000000..10c02103aedd --- /dev/null +++ b/project/MimaBuild.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. + */ + +import sbt._ +import sbt.Keys.version + +import com.typesafe.tools.mima.core._ +import com.typesafe.tools.mima.core.MissingClassProblem +import com.typesafe.tools.mima.core.MissingTypesProblem +import com.typesafe.tools.mima.core.ProblemFilters._ +import com.typesafe.tools.mima.plugin.MimaKeys.{mimaBinaryIssueFilters, mimaPreviousArtifacts} +import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings + + +object MimaBuild { + + def excludeMember(fullName: String) = Seq( + ProblemFilters.exclude[MissingMethodProblem](fullName), + // Sometimes excluded methods have default arguments and + // they are translated into public methods/fields($default$) in generated + // bytecode. It is not possible to exhaustively list everything. + // But this should be okay. + ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$2"), + ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$1"), + ProblemFilters.exclude[MissingFieldProblem](fullName), + ProblemFilters.exclude[IncompatibleResultTypeProblem](fullName), + ProblemFilters.exclude[IncompatibleMethTypeProblem](fullName), + ProblemFilters.exclude[IncompatibleFieldTypeProblem](fullName) + ) + + // Exclude a single class + def excludeClass(className: String) = Seq( + ProblemFilters.exclude[Problem](className + ".*"), + ProblemFilters.exclude[MissingClassProblem](className), + ProblemFilters.exclude[MissingTypesProblem](className) + ) + + // Exclude a Spark class, that is in the package org.apache.spark + def excludeSparkClass(className: String) = { + excludeClass("org.apache.spark." + className) + } + + // Exclude a Spark package, that is in the package org.apache.spark + def excludeSparkPackage(packageName: String) = { + ProblemFilters.exclude[Problem]("org.apache.spark." + packageName + ".*") + } + + def ignoredABIProblems(base: File, currentSparkVersion: String) = { + + // Excludes placed here will be used for all Spark versions + val defaultExcludes = Seq() + + // Read package-private excludes from file + val classExcludeFilePath = file(base.getAbsolutePath + "/.generated-mima-class-excludes") + val memberExcludeFilePath = file(base.getAbsolutePath + "/.generated-mima-member-excludes") + + val ignoredClasses: Seq[String] = + if (!classExcludeFilePath.exists()) { + Seq() + } else { + IO.read(classExcludeFilePath).split("\n") + } + + val ignoredMembers: Seq[String] = + if (!memberExcludeFilePath.exists()) { + Seq() + } else { + IO.read(memberExcludeFilePath).split("\n") + } + + defaultExcludes ++ ignoredClasses.flatMap(excludeClass) ++ + ignoredMembers.flatMap(excludeMember) ++ MimaExcludes.excludes(currentSparkVersion) + } + + def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { + val organization = "org.apache.spark" + val previousSparkVersion = "2.4.0" + val project = projectRef.project + val fullId = "spark-" + project + "_2.12" + mimaDefaultSettings ++ + Seq(mimaPreviousArtifacts := Set(organization % fullId % previousSparkVersion), + mimaBinaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value)) + } + +} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala new file mode 100644 index 000000000000..57fbb125dc47 --- /dev/null +++ b/project/MimaExcludes.scala @@ -0,0 +1,1698 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.typesafe.tools.mima.core._ +import com.typesafe.tools.mima.core.ProblemFilters._ + +/** + * Additional excludes for checking of Spark's binary compatibility. + * + * This acts as an official audit of cases where we excluded other classes. Please use the narrowest + * possible exclude here. MIMA will usually tell you what exclude to use, e.g.: + * + * ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.take") + * + * It is also possible to exclude Spark classes and packages. This should be used sparingly: + * + * MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") + * + * For a new Spark version, please update MimaBuild.scala to reflect the previous version. + */ +object MimaExcludes { + + // Exclude rules for 3.1.x + lazy val v31excludes = v30excludes ++ Seq( + // [SPARK-31077] Remove ChiSqSelector dependency on mllib.ChiSqSelectorModel + // private constructor + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.this"), + // [SPARK-31127] Implement abstract Selector + // org.apache.spark.ml.feature.ChiSqSelectorModel type hierarchy change + // before: class ChiSqSelector extends Estimator with ChiSqSelectorParams + // after: class ChiSqSelector extends PSelector + // false positive, no binary incompatibility + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.ChiSqSelector"), + + //[SPARK-31840] Add instance weight support in LogisticRegressionSummary + // weightCol in org.apache.spark.ml.classification.LogisticRegressionSummary is present only in current version + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightCol") + ) + + // 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"), + + // [SPARK-28980][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 + 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"), + + // [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"), + + // [SPARK-26580][SQL][ML][FOLLOW-UP] Throw exception when use untyped UDF by default + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.UnaryTransformer.this"), + + // [SPARK-27090][CORE] Removing old LEGACY_DRIVER_IDENTIFIER ("") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.LEGACY_DRIVER_IDENTIFIER"), + + // [SPARK-25838] Remove formatVersion from Saveable + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.DistributedLDAModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.LocalLDAModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeansModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.PowerIterationClusteringModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.GaussianMixtureModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.recommendation.MatrixFactorizationModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.ChiSqSelectorModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.Word2VecModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.SVMModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.LogisticRegressionModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.NaiveBayesModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.Saveable.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.fpm.FPGrowthModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.fpm.PrefixSpanModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.IsotonicRegressionModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.RidgeRegressionModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.LassoModel.formatVersion"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.LinearRegressionModel.formatVersion"), + + // [SPARK-26132] Remove support for Scala 2.11 in Spark 3.0.0 + ProblemFilters.exclude[DirectAbstractMethodProblem]("scala.concurrent.Future.transformWith"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("scala.concurrent.Future.transform"), + + // [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.deploy.security.HiveDelegationTokenProvider"), + + // [SPARK-26329][CORE] Faster polling of executor memory metrics. + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.apply"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.copy$default$6"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.this"), + + // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), + 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"), + + // [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"), + + // [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.launcher.SparkAppHandle.getError"), + + // [SPARK-25867] Remove KMeans computeCost + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), + + // [SPARK-26127] Remove deprecated setters from tree regression and classification models + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setSeed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMinInfoGain"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setCacheNodeIds"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setCheckpointInterval"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMaxDepth"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setImpurity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMaxMemoryInMB"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMaxBins"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.setMinInstancesPerNode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setSeed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMinInfoGain"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setSubsamplingRate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxIter"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setCacheNodeIds"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setCheckpointInterval"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxDepth"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setImpurity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxMemoryInMB"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setStepSize"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMaxBins"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setMinInstancesPerNode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.setFeatureSubsetStrategy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setSeed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMinInfoGain"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setSubsamplingRate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setCacheNodeIds"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setCheckpointInterval"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMaxDepth"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setImpurity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMaxMemoryInMB"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setFeatureSubsetStrategy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMaxBins"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setMinInstancesPerNode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setNumTrees"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setSeed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMinInfoGain"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setCacheNodeIds"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setCheckpointInterval"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMaxDepth"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setImpurity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMaxMemoryInMB"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMaxBins"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.setMinInstancesPerNode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setSeed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMinInfoGain"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setSubsamplingRate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxIter"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setCacheNodeIds"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setCheckpointInterval"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxDepth"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setImpurity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxMemoryInMB"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setStepSize"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMaxBins"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setMinInstancesPerNode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.setFeatureSubsetStrategy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setSeed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMinInfoGain"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setSubsamplingRate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setCacheNodeIds"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setCheckpointInterval"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMaxDepth"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setImpurity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMaxMemoryInMB"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMaxBins"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setMinInstancesPerNode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setNumTrees"), + + // [SPARK-26090] Resolve most miscellaneous deprecation and build warnings for Spark 3 + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.stat.test.BinarySampleBeanInfo"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.LabeledPointBeanInfo"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.LabeledPointBeanInfo"), + + // [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-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"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleBytesWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleWriteTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.fMeasure"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.recall"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.precision"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.util.MLWriter.context"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.util.MLReader.context"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.util.GeneralMLWriter.context"), + + // [SPARK-25737] Remove JavaSparkContextVarargsWorkaround + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.api.java.JavaSparkContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.union"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.union"), + + // [SPARK-16775] Remove deprecated accumulator v1 APIs + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulable"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulator"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulator$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulableParam"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$FloatAccumulatorParam$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$DoubleAccumulatorParam$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$LongAccumulatorParam$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$IntAccumulatorParam$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulable"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulableCollection"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulator"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.LegacyAccumulatorWrapper"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.intAccumulator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.accumulable"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.doubleAccumulator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.accumulator"), + + // [SPARK-24109] Remove class SnappyOutputStreamWrapper + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.SnappyCompressionCodec.version"), + + // [SPARK-19287] JavaPairRDD flatMapValues requires function returning Iterable, not Iterator + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.api.java.JavaPairRDD.flatMapValues"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaPairDStream.flatMapValues"), + + // [SPARK-25680] SQL execution listener shouldn't happen on execution thread + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.util.ExecutionListenerManager.clone"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.util.ExecutionListenerManager.this"), + + // [SPARK-25862][SQL] Remove rangeBetween APIs introduced in SPARK-21608 + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.unboundedFollowing"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.unboundedPreceding"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.currentRow"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.expressions.Window.rangeBetween"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.expressions.WindowSpec.rangeBetween"), + + // [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.nextCredentialRenewalTime"), + + // [SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.OneHotEncoder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.feature.OneHotEncoderEstimator$"), + + // [SPARK-30329][ML] add iterator/foreach methods for Vectors + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Vector.activeIterator"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.activeIterator"), + + // [SPARK-26141] Enable custom metrics implementation in shuffle write + // Following are Java private classes + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.TimeTrackingOutputStream.this"), + + // [SPARK-26139] Implement shuffle write metrics in SQL + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ShuffleDependency.this"), + + // [SPARK-26362][CORE] Remove 'spark.driver.allowMultipleContexts' to disallow multiple creation of SparkContexts + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.setActiveContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.markPartiallyConstructed"), + + // [SPARK-26457] Show hadoop configurations in HistoryServer environment tab + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this"), + + // [SPARK-30144][ML] Make MultilayerPerceptronClassificationModel extend MultilayerPerceptronParams + 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) => + !cls.fullName.startsWith("org.apache.spark.sql.sources.v2") + case _ => true + }, + + // [SPARK-27521][SQL] Move data source v2 to catalyst module + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarBatch"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ArrowColumnVector"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarRow"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarArray"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnarMap"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.vectorized.ColumnVector"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThanOrEqual"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringEndsWith"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThanOrEqual$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.In$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Not"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNotNull"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThan"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThanOrEqual"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualNullSafe$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThan$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.In"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.And"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringStartsWith$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualNullSafe"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringEndsWith$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThanOrEqual$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Not$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNull$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LessThan$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNotNull$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Or"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualTo$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.GreaterThan"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringContains"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Filter"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.IsNull"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.EqualTo"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.And$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Or$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringStartsWith"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.StringContains$"), + + // [SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction) + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction$"), + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.sql.expressions.UserDefinedFunction"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.inputTypes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullableTypes_="), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.dataType"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.f"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.this"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNonNullable"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNonNullable"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullable"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.nullable"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNondeterministic"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.asNondeterministic"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.deterministic"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.deterministic"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.apply"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.apply"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.withName"), + ProblemFilters.exclude[ReversedAbstractMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.withName"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$2"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$1"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.UserDefinedFunction.copy$default$3"), + + // [SPARK-11215][ML] Add multiple columns support to StringIndexer + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.StringIndexer.validateAndTransformSchema"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.StringIndexerModel.validateAndTransformSchema"), + + // [SPARK-26616][MLlib] Expose document frequency in IDFModel + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"), + + // [SPARK-28199][SS] Remove deprecated ProcessingTime + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime$"), + + // [SPARK-25382][SQL][PYSPARK] Remove ImageSchema.readImages in 3.0 + 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"), + + // [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"), + + // [SPARK-29348] Add observable metrics. + 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"), + + // [SPARK-29543][SS][UI] Init structured streaming ui + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStartedEvent.this"), + + // [SPARK-30667][CORE] Add allGather method to BarrierTaskContext + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.RequestToSync") + ) + + // Exclude rules for 2.4.x + lazy val v24excludes = v23excludes ++ Seq( + // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$"), + + // [SPARK-25248] add package private methods to TaskContext + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markTaskFailed"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markInterrupted"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.fetchFailed"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markTaskCompleted"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getLocalProperties"), + + // [SPARK-10697][ML] Add lift to Association rules + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.fpm.FPGrowthModel.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.fpm.AssociationRules#Rule.this"), + + // [SPARK-24296][CORE] Replicate large blocks as a stream. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockRpcServer.this"), + // [SPARK-23528] Add numIter to ClusteringSummary + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.ClusteringSummary.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.KMeansSummary.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.BisectingKMeansSummary.this"), + // [SPARK-6237][NETWORK] Network-layer changes to allow stream upload + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockRpcServer.receive"), + + // [SPARK-20087][CORE] Attach accumulators / metrics to 'TaskKilled' end reason + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.this"), + + // [SPARK-22941][core] Do not exit JVM when submit fails with in-process launcher. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.printWarning"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.parseSparkConfProperty"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.printVersionAndExit"), + + // [SPARK-23412][ML] Add cosine distance measure to BisectingKmeans + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasDistanceMeasure.org$apache$spark$ml$param$shared$HasDistanceMeasure$_setter_$distanceMeasure_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasDistanceMeasure.getDistanceMeasure"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasDistanceMeasure.distanceMeasure"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel#SaveLoadV1_0.load"), + + // [SPARK-20659] Remove StorageStatus, or make it private + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.totalOffHeapStorageMemory"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.usedOffHeapStorageMemory"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.usedOnHeapStorageMemory"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkExecutorInfo.totalOnHeapStorageMemory"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.getExecutorStorageStatus"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.numBlocks"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.numRddBlocks"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.containsBlock"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.rddBlocksById"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.numRddBlocksById"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.memUsedByRdd"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.cacheSize"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.rddStorageLevel"), + + // [SPARK-23455][ML] Default Params in ML should be saved separately in metadata + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.paramMap"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$paramMap_="), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.defaultParamMap"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$defaultParamMap_="), + + // [SPARK-7132][ML] Add fit with validation set to spark.ml GBT + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), + + // [SPARK-23042] Use OneHotEncoderModel to encode labels in MultilayerPerceptronClassifier + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.classification.LabelConverter"), + + // [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getDateOfNextUpdate"), + + // [SPARK-23366] Improve hot reading path in ReadAheadInputStream + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.ReadAheadInputStream.this"), + + // [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.addJarToClasspath"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.mergeFileLists"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment$default$2"), + + // Data Source V2 API changes + // TODO: they are unstable APIs and should not be tracked by mima. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.ReadSupportWithSchema"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.createDataReaderFactories"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.createBatchDataReaderFactories"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.planBatchInputPartitions"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanUnsafeRow"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader.createDataReaderFactories"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader.planInputPartitions"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsPushDownCatalystFilters"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataReader"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.getStatistics"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.estimateStatistics"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataReaderFactory"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.ContinuousDataReader"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.SupportsWriteInternalRow"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createDataWriter"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createDataWriter"), + + // Changes to HasRawPredictionCol. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.rawPredictionCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.org$apache$spark$ml$param$shared$HasRawPredictionCol$_setter_$rawPredictionCol_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.getRawPredictionCol"), + + // [SPARK-15526][ML][FOLLOWUP] Make JPMML provided scope to avoid including unshaded JARs + (problem: Problem) => problem match { + case MissingClassProblem(cls) => + !cls.fullName.startsWith("org.sparkproject.jpmml") && + !cls.fullName.startsWith("org.sparkproject.dmg.pmml") && + !cls.fullName.startsWith("org.spark_project.jpmml") && + !cls.fullName.startsWith("org.spark_project.dmg.pmml") + case _ => true + } + ) + + // Exclude rules for 2.3.x + lazy val v23excludes = v22excludes ++ Seq( + // [SPARK-22897] Expose stageAttemptId in TaskContext + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.stageAttemptNumber"), + + // SPARK-22789: Map-only continuous processing execution + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.startQuery$default$8"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.startQuery$default$6"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.startQuery$default$9"), + + // SPARK-22372: Make cluster submission use SparkApplication. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getSecretKeyFromUserCredentials"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.isYarnMode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getCurrentUserCredentials"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.addSecretKeyToUserCredentials"), + + // SPARK-18085: Better History Server scalability for many / large applications + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ExecutorSummary.executorLogs"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.getSparkUI"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.env.EnvironmentListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.exec.ExecutorsListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.storage.StorageListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.StorageStatusListener"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorStageSummary.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.JobData.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkStatusTracker.this"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.jobs.JobProgressListener"), + + // [SPARK-20495][SQL] Add StorageLevel to cacheTable API + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.cacheTable"), + + // [SPARK-19937] Add remote bytes read to disk. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetricDistributions.this"), + + // [SPARK-21276] Update lz4-java to the latest (v1.4.0) + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.io.LZ4BlockInputStream"), + + // [SPARK-17139] Add model summary for MultinomialLogisticRegression + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictionCol"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.labels"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.truePositiveRateByLabel"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.falsePositiveRateByLabel"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.precisionByLabel"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.recallByLabel"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.fMeasureByLabel"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.accuracy"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedTruePositiveRate"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedFalsePositiveRate"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedRecall"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedPrecision"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.weightedFMeasure"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.asBinary"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.org$apache$spark$ml$classification$LogisticRegressionSummary$$multiclassMetrics"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.org$apache$spark$ml$classification$LogisticRegressionSummary$_setter_$org$apache$spark$ml$classification$LogisticRegressionSummary$$multiclassMetrics_="), + + // [SPARK-14280] Support Scala 2.12 + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.FutureAction.transformWith"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.FutureAction.transform"), + + // [SPARK-21087] CrossValidator, TrainValidationSplit expose sub models after fitting: Scala + ProblemFilters.exclude[FinalClassProblem]("org.apache.spark.ml.tuning.CrossValidatorModel$CrossValidatorModelWriter"), + ProblemFilters.exclude[FinalClassProblem]("org.apache.spark.ml.tuning.TrainValidationSplitModel$TrainValidationSplitModelWriter"), + + // [SPARK-21728][CORE] Allow SparkSubmit to use Logging + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.downloadFileList"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.downloadFile"), + + // [SPARK-21714][CORE][YARN] Avoiding re-uploading remote resources in yarn client mode + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment"), + + // [SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0 + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.network.util.AbstractFileRegion.transfered"), + + // [SPARK-20643][CORE] Add listener implementation to collect app state + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$5"), + + // [SPARK-20648][CORE] Port JobsTab and StageTab to the new UI backend + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$12"), + + // [SPARK-21462][SS] Added batchId to StreamingQueryProgress.json + // [SPARK-21409][SS] Expose state store memory usage in SQL metrics and progress updates + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StateOperatorProgress.this"), + + // [SPARK-22278][SS] Expose current event time watermark and current processing time in GroupState + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.GroupState.getCurrentWatermarkMs"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.GroupState.getCurrentProcessingTimeMs"), + + // [SPARK-20542][ML][SQL] Add an API to Bucketizer that can bin multiple columns + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasOutputCols.org$apache$spark$ml$param$shared$HasOutputCols$_setter_$outputCols_="), + + // [SPARK-18619][ML] Make QuantileDiscretizer/Bucketizer/StringIndexer/RFormula inherit from HasHandleInvalid + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.Bucketizer.getHandleInvalid"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.StringIndexer.getHandleInvalid"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.getHandleInvalid"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.StringIndexerModel.getHandleInvalid") + ) + + // Exclude rules for 2.2.x + lazy val v22excludes = v21excludes ++ Seq( + // [SPARK-20355] Add per application spark version on the history server headerpage + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), + + // [SPARK-19652][UI] Do auth checks for REST API access. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.withSparkUI"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.status.api.v1.UIRootFromServletContext"), + + // [SPARK-18663][SQL] Simplify CountMinSketch aggregate implementation + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.CountMinSketch.toByteArray"), + + // [SPARK-18949] [SQL] Add repairTable API to Catalog + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.recoverPartitions"), + + // [SPARK-18537] Add a REST api to spark streaming + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.streaming.scheduler.StreamingListener.onStreamingStarted"), + + // [SPARK-19148][SQL] do not expose the external table concept in Catalog + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createTable"), + + // [SPARK-14272][ML] Add logLikelihood in GaussianMixtureSummary + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.GaussianMixtureSummary.this"), + + // [SPARK-19267] Fetch Failure handling robust to user error handling + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.setFetchFailed"), + + // [SPARK-19069] [CORE] Expose task 'status' and 'duration' in spark history server REST API. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$10"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$11"), + + // [SPARK-17161] Removing Python-friendly constructors not needed + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.OneVsRestModel.this"), + + // [SPARK-19820] Allow reason to be specified to task kill + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.TaskKilled$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.countTowardsTaskFailures"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.toErrorString"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.TaskKilled.toString"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.killTaskIfInterrupted"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getKillReason"), + + // [SPARK-19876] Add one time trigger, and improve Trigger APIs + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.sql.streaming.Trigger"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.streaming.ProcessingTime"), + + // [SPARK-17471][ML] Add compressed method to ML matrices + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.compressed"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.compressedColMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.compressedRowMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.isRowMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.isColMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSparseSizeInBytes"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDense"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparse"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseRowMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseRowMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseColMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getDenseSizeInBytes"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseColMajor"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseMatrix"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseMatrix"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSizeInBytes"), + + // [SPARK-18693] Added weightSum to trait MultivariateStatisticalSummary + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.stat.MultivariateStatisticalSummary.weightSum") + ) ++ Seq( + // [SPARK-17019] Expose on-heap and off-heap memory usage in various places + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.StorageStatus.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.RDDDataDistribution.this") + ) + + // Exclude rules for 2.1.x + lazy val v21excludes = v20excludes ++ { + Seq( + // [SPARK-17671] Spark 2.0 history server summary page is slow even set spark.history.ui.maxApplications + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.deploy.history.HistoryServer.getApplicationList"), + // [SPARK-14743] Improve delegation token handling in secure cluster + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal"), + // [SPARK-16199][SQL] Add a method to list the referenced columns in data source Filter + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.Filter.references"), + // [SPARK-16853][SQL] Fixes encoder error in DataSet typed select + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.select"), + // [SPARK-16967] Move Mesos to Module + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkMasterRegex.MESOS_REGEX"), + // [SPARK-16240] ML persistence backward compatibility for LDA + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.clustering.LDA$"), + // [SPARK-17717] Add Find and Exists method to Catalog. + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getDatabase"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getTable"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getFunction"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.databaseExists"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"), + + // [SPARK-17731][SQL][Streaming] Metrics for structured streaming + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryStarted"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), + + // [SPARK-18516][SQL] Split state and progress in streaming + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.SourceStatus"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.SinkStatus"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.sinkStatus"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.sourceStatuses"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQuery.id"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.lastProgress"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.recentProgress"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.id"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryManager.get"), + + // [SPARK-17338][SQL] add global temp view + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), + + // [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.aggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.getAggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.org$apache$spark$ml$param$shared$HasAggregationDepth$_setter_$aggregationDepth_="), + + // [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.TaskInfo.accumulables"), + + // [SPARK-18657] Add StreamingQuery.runId + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.runId"), + + // [SPARK-18694] Add StreamingQuery.explain and exception to Python and fix StreamingQueryException + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryException$"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.startOffset"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.endOffset"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryException.query") + ) + } + + // Exclude rules for 2.0.x + lazy val v20excludes = { + Seq( + ProblemFilters.exclude[Problem]("org.apache.spark.rpc.*"), + ProblemFilters.exclude[Problem]("org.spark-project.jetty.*"), + ProblemFilters.exclude[Problem]("org.spark_project.jetty.*"), + ProblemFilters.exclude[Problem]("org.sparkproject.jetty.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.internal.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.unused.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.unsafe.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.memory.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.util.collection.unsafe.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.sql.catalyst.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.sql.execution.*"), + ProblemFilters.exclude[Problem]("org.apache.spark.sql.internal.*"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.status.api.v1.ApplicationAttemptInfo.$default$5"), + // SPARK-14042 Add custom coalescer support + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"), + // SPARK-15532 Remove isRootContext flag from SQLContext. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.isRootContext"), + // SPARK-12600 Remove SQL deprecated methods + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.applySchema"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.parquetFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jdbc"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.load"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.dialectClassName"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.getSQLDialect"), + // SPARK-13664 Replace HadoopFsRelation with FileFormat + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.LibSVMRelation"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelationProvider"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache"), + // SPARK-15543 Rename DefaultSources to make them more self-describing + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.DefaultSource") + ) ++ Seq( + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory"), + // SPARK-14358 SparkListener from trait to abstract class + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.addSparkListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.JavaSparkListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.SparkFirehoseListener"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.scheduler.SparkListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.jobs.JobProgressListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.exec.ExecutorsListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.env.EnvironmentListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.storage.StorageListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.StorageStatusListener") + ) ++ + Seq( + // SPARK-3369 Fix Iterable/Iterator in Java API + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapFunction2.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapFunction2.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.PairFlatMapFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.PairFlatMapFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.CoGroupFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.CoGroupFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.MapPartitionsFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.MapPartitionsFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapGroupsFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapGroupsFunction.call") + ) ++ + Seq( + // [SPARK-6429] Implement hashCode and equals together + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Partition.org$apache$spark$Partition$$super=uals") + ) ++ + Seq( + // SPARK-4819 replace Guava Optional + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getCheckpointDir"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getSparkHome"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner") + ) ++ + Seq( + // SPARK-12481 Remove Hadoop 1.x + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.mapred.SparkHadoopMapRedUtil"), + // SPARK-12615 Remove deprecated APIs in core + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.$default$6"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.numericRDDToDoubleRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intToIntWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.writableWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToAsyncRDDActions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.boolToBoolWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longToLongWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToOrderedRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.booleanWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringToText"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleToDoubleWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToSequenceFileRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesToBytesWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatToFloatWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions$default$4"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addOnCompleteCallback"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.runningLocally"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.attemptId"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.defaultMinSplits"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.runJob"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.runJob"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.tachyonFolderName"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.initLocalProperties"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearJars"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearFiles"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith$default$2"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.toArray"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith$default$2"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.filterWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.foreachWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit$default$2"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.SequenceFileRDDFunctions.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.splits"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.toArray"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.defaultMinSplits"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearJars"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearFiles"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.externalBlockStoreFolderName"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockManager"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore") + ) ++ Seq( + // SPARK-12149 Added new fields to ExecutorSummary + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") + ) ++ + // SPARK-12665 Remove deprecated and unused classes + Seq( + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.graphx.GraphKryoRegistrator"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") + ) ++ Seq( + // SPARK-12591 Register OpenHashMapBasedStateMap for Kryo + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoInputDataInputBridge"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoOutputDataOutputBridge") + ) ++ Seq( + // SPARK-12510 Refactor ActorReceiver to support Java + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") + ) ++ Seq( + // SPARK-12895 Implement TaskMetrics using accumulators + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.internalMetricsToAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectInternalAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectAccumulators") + ) ++ Seq( + // SPARK-12896 Send only accumulator updates to driver, not TaskMetrics + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Accumulator.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.initialValue") + ) ++ Seq( + // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") + ) ++ Seq( + // SPARK-12689 Migrate DDL parsing to the newly absorbed parser + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLException"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.ddlParser") + ) ++ Seq( + // SPARK-7799 Add "streaming-akka" project + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$6"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$5"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$4"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$3"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.actorStream"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.streaming.zeromq.ZeroMQReceiver"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver$Supervisor") + ) ++ Seq( + // SPARK-12348 Remove deprecated Streaming APIs. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.dstream.DStream.foreach"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions$default$4"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.awaitTermination"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.networkStream"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.api.java.JavaStreamingContextFactory"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.awaitTermination"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.sc"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreachRDD"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreach"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.getOrCreate") + ) ++ Seq( + // SPARK-12847 Remove StreamingListenerBus and post all Streaming events to the same thread as Spark events + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus") + ) ++ Seq( + // SPARK-11622 Make LibSVMRelation extends HadoopFsRelation and Add LibSVMOutputWriter + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.source.libsvm.DefaultSource"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.source.libsvm.DefaultSource.createRelation") + ) ++ Seq( + // SPARK-6363 Make Scala 2.11 the default Scala version + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.cleanup"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metadataCleaner"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnDriverEndpoint"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint") + ) ++ Seq( + // SPARK-7889 + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.org$apache$spark$deploy$history$HistoryServer$@tachSparkUI"), + // SPARK-13296 + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.UDFRegistration.register"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction$") + ) ++ Seq( + // SPARK-12995 Remove deprecated APIs in graphx + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.lib.SVDPlusPlus.runSVDPlusPlus"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets$default$3"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.impl.GraphImpl.mapReduceTriplets") + ) ++ Seq( + // SPARK-13426 Remove the support of SIMR + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkMasterRegex.SIMR_REGEX") + ) ++ Seq( + // SPARK-13413 Remove SparkContext.metricsSystem/schedulerBackend_ setter + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metricsSystem"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.schedulerBackend_=") + ) ++ Seq( + // SPARK-13220 Deprecate yarn-client and yarn-cluster mode + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") + ) ++ Seq( + // SPARK-13465 TaskContext. + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addTaskFailureListener") + ) ++ Seq ( + // SPARK-7729 Executor which has been killed should also be displayed on Executor Tab + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") + ) ++ Seq( + // SPARK-13526 Move SQLContext per-session states to new class + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.sql.UDFRegistration.this") + ) ++ Seq( + // [SPARK-13486][SQL] Move SQLConf into an internal package + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry$") + ) ++ Seq( + //SPARK-11011 UserDefinedType serialization should be strongly typed + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), + // SPARK-12073: backpressure rate controller consumes events preferentially from lagging partitions + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.KafkaTestUtils.createTopic"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.DirectKafkaInputDStream.maxMessagesPerPartition") + ) ++ Seq( + // [SPARK-13244][SQL] Migrates DataFrame to Dataset + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.tables"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.sql"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.baseRelationToDataFrame"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.table"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrame.apply"), + + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.LegacyFunctions"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.localSeqToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.stringRddToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.rddToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.longRddToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.intRddToDataFrameHolder"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.GroupedDataset"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.subtract"), + + // [SPARK-14451][SQL] Move encoder definition into Aggregator interface + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.toColumn"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.bufferEncoder"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.outputEncoder"), + + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MultilabelMetrics.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions") + ) ++ Seq( + // [SPARK-13686][MLLIB][STREAMING] Add a constructor parameter `reqParam` to (Streaming)LinearRegressionWithSGD + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD.this") + ) ++ Seq( + // SPARK-15250 Remove deprecated json API in DataFrameReader + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameReader.json") + ) ++ Seq( + // SPARK-13920: MIMA checks should apply to @Experimental and @DeveloperAPI APIs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineCombinersByKey"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineValuesByKey"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.run"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.runJob"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.actorSystem"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.cacheManager"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getConfigurationFromJobContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTaskAttemptIDFromTaskAttemptContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.newConfiguration"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback_="), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.setBytesReadCallback"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.updateBytesRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decFetchWaitTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decLocalBlocksFetched"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRecordsRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBlocksFetched"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBytesRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleBytesWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleWriteTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleBytesWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleWriteTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.setShuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.PCAModel.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.taskMetrics"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.TaskInfo.attempt"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.ExperimentalMethods.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUDF"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUdf"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.cumeDist"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.denseRank"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.inputFileName"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.isNaN"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.percentRank"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.rowNumber"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.sparkPartitionId"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.externalBlockStoreSize"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsedByRdd"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatusListener.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.scheduler.BatchInfo.streamIdToNumRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.storageStatusList"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.storage.StorageListener.storageStatusList"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.InputMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.OutputMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Pipeline.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PipelineModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transformImpl"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.extractLabeledPoints"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.GBTClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayes.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRest.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRestModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.RandomForestClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeans.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logLikelihood"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logPerplexity"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.RegressionEvaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Binarizer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Bucketizer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelector.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.HashingTF.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDF.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDFModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IndexToString.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Interaction.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScaler.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCA.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCAModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormula.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormulaModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.SQLTransformer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScaler.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StopWordsRemover.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorAssembler.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorSlicer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2Vec.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALS.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.GBTRegressor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.extractWeightedLabeledPoints"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.extractWeightedLabeledPoints"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegression.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionTrainingSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidator.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidatorModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplit.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplitModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.BinaryClassificationMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.RegressionMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameWriter.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.broadcast"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.callUDF"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.InsertableRelation.insert"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.fMeasureByThreshold"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.pr"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.precisionByThreshold"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.recallByThreshold"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.roc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.describeTopics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.getVectors"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.itemFactors"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.userFactors"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.residuals"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.name"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.value"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.drop"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.fill"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.replace"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.jdbc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.json"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.load"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.orc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.parquet"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.table"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.text"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.crosstab"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.freqItems"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.sampleBy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.emptyDataFrame"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.range"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.functions.udf"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.JobLogger"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorHelper"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics$"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics$"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics$"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.functions$"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Estimator.fit"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Predictor.train"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Transformer.transform"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListener.onOtherEvent"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.InsertableRelation.insert") + ) ++ Seq( + // [SPARK-13926] Automatically use Kryo serializer when shuffling RDDs with simple types + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ShuffleDependency.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ShuffleDependency.serializer"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.Serializer$") + ) ++ Seq( + // SPARK-13927: add row/column iterator to local matrices + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.rowIter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.colIter") + ) ++ Seq( + // SPARK-13948: MiMa Check should catch if the visibility change to `private` + // TODO(josh): Some of these may be legitimate incompatibilities; we should follow up before the 2.0.0 release + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.toDS"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.askTimeout"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.lookupTimeout"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.UnaryTransformer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.select"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.toDF"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Logging.initializeLogIfNecessary"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerEvent.logEvent"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance") + ) ++ Seq( + // [SPARK-14014] Replace existing analysis.Catalog with SessionCatalog + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.this") + ) ++ Seq( + // [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Logging"), + (problem: Problem) => problem match { + case MissingTypesProblem(_, missing) + if missing.map(_.fullName).sameElements(Seq("org.apache.spark.Logging")) => false + case _ => true + } + ) ++ Seq( + // [SPARK-13990] Automatically pick serializer when caching RDDs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.uploadBlock") + ) ++ Seq( + // [SPARK-14089][CORE][MLLIB] Remove methods that has been deprecated since 1.1, 1.2, 1.3, 1.4, and 1.5 + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.getThreadLocal"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeReduce"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeAggregate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.tree.configuration.Strategy.defaultStategy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.saveLabeledData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLabeledData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.optimization.LBFGS.setMaxNumIterations"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.setScoreCol") + ) ++ Seq( + // [SPARK-14205][SQL] remove trait Queryable + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.Dataset") + ) ++ Seq( + // [SPARK-11262][ML] Unit test for gradient, loss layers, memory management + // for multilayer perceptron. + // This class is marked as `private`. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.ann.SoftmaxFunction") + ) ++ Seq( + // [SPARK-13674][SQL] Add wholestage codegen support to Sample + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.util.random.PoissonSampler.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.random.PoissonSampler.this") + ) ++ Seq( + // [SPARK-13430][ML] moved featureCol from LinearRegressionModelSummary to LinearRegressionSummary + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this") + ) ++ Seq( + // [SPARK-14437][Core] Use the address that NettyBlockTransferService listens to create BlockManagerId + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.this") + ) ++ Seq( + // [SPARK-13048][ML][MLLIB] keepLastCheckpoint option for LDA EM optimizer + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.DistributedLDAModel.this") + ) ++ Seq( + // [SPARK-14475] Propagate user-defined context from driver to executors + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getLocalProperty"), + // [SPARK-14617] Remove deprecated APIs in TaskMetrics + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.InputMetrics$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.OutputMetrics$"), + // [SPARK-14628] Simplify task metrics by always tracking read/write metrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.readMethod"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.writeMethod") + ) ++ Seq( + // SPARK-14628: Always track input/output/shuffle metrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.totalBlocksFetched"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.inputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.outputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleWriteMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleReadMetrics"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.inputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.outputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleWriteMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleReadMetrics"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") + ) ++ Seq( + // SPARK-13643: Move functionality from SQLContext to SparkSession + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.getSchema") + ) ++ Seq( + // [SPARK-14407] Hides HadoopFsRelation related data source API into execution package + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriter"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriterFactory") + ) ++ Seq( + // SPARK-14734: Add conversions between mllib and ml Vector, Matrix types + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asML"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asML") + ) ++ Seq( + // SPARK-14704: Create accumulators in TaskMetrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.this") + ) ++ Seq( + // SPARK-14861: Replace internal usages of SQLContext with SparkSession + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.clustering.LocalLDAModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.clustering.DistributedLDAModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.clustering.LDAModel.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]( + "org.apache.spark.ml.clustering.LDAModel.sqlContext"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.sql.Dataset.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.sql.DataFrameReader.this") + ) ++ Seq( + // SPARK-14542 configurable buffer size for pipe RDD + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.pipe"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.pipe") + ) ++ Seq( + // [SPARK-4452][Core]Shuffle data structures can starve others on the same thread for memory + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.util.collection.Spillable") + ) ++ Seq( + // [SPARK-14952][Core][ML] Remove methods deprecated in 1.6 + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.input.PortableDataStream.close"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.weights"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionModel.weights") + ) ++ Seq( + // [SPARK-10653] [Core] Remove unnecessary things from SparkEnv + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.sparkFilesDir"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.blockTransferService") + ) ++ Seq( + // SPARK-14654: New accumulator API + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ExceptionFailure$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.metrics"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.remoteBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.totalBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") + ) ++ Seq( + // [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.getOldDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.estimatedDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.topicsMatrix"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.clusterCenters"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.decodeLabel"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.encodeLabeledPoint"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.weights"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.predict"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.theta"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.pi"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.probability2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.getScalingVec"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.setScalingVec"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PCAModel.pc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMax"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMin"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.IDFModel.idf"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.mean"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.std"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predictQuantiles"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.boundaries"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.this") + ) ++ Seq( + // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Private"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") + ) ++ Seq( + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asBreeze"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asBreeze") + ) ++ Seq( + // [SPARK-15914] Binary compatibility is broken since consolidation of Dataset and DataFrame + // in Spark 2.0. However, source level compatibility is still maintained. + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.load"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonFile"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") + ) ++ Seq( + // SPARK-17096: Improve exception string reported through the StreamingQueryListener + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.stackTrace"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.this") + ) ++ Seq( + // SPARK-17406 limit timeline executor events + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorIdToData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksActive"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksComplete"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksFailed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleWrite"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToDuration"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputBytes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToLogUrls"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputBytes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTotalCores"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksMax"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToJvmGCTime") + ) ++ Seq( + // [SPARK-17163] Unify logistic regression interface. Private constructor has new signature. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this") + ) ++ Seq( + // [SPARK-17498] StringIndexer enhancement for handling unseen labels + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.StringIndexer"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.feature.StringIndexerModel") + ) ++ Seq( + // [SPARK-17365][Core] Remove/Kill multiple executors together to reduce RPC call time + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.SparkContext") + ) ++ Seq( + // [SPARK-12221] Add CPU time to metrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") + ) ++ Seq( + // [SPARK-18481] ML 2.1 QA: Remove deprecated methods for ML + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.PipelineStage.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.param.JavaParams.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.param.Params.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegression.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassifier.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.numTrees"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.setLabelCol"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.evaluation.Evaluator.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressor.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.validateParams"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.model"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.numTrees"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.RandomForestClassifier"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.GBTClassifier"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.classification.GBTClassificationModel"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.RandomForestRegressor"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.GBTRegressor"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.regression.GBTRegressionModel"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.getNumTrees"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.getNumTrees"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.numTrees"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setFeatureSubsetStrategy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.numTrees"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy") + ) ++ Seq( + // [SPARK-21680][ML][MLLIB]optimzie Vector coompress + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.toSparseWithSize"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Vector.toSparseWithSize") + ) ++ Seq( + // [SPARK-3181][ML]Implement huber loss for LinearRegression. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.org$apache$spark$ml$param$shared$HasLoss$_setter_$loss_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.getLoss"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.loss") + ) + } + + def excludes(version: String) = version match { + case v if v.startsWith("3.1") => v31excludes + case v if v.startsWith("3.0") => v30excludes + case v if v.startsWith("2.4") => v24excludes + case v if v.startsWith("2.3") => v23excludes + case v if v.startsWith("2.2") => v22excludes + case v if v.startsWith("2.1") => v21excludes + case v if v.startsWith("2.0") => v20excludes + case _ => Seq() + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala new file mode 100644 index 000000000000..eb12f2f1f6ab --- /dev/null +++ b/project/SparkBuild.scala @@ -0,0 +1,1058 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 java.io._ +import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files +import java.util.Locale + +import scala.io.Source +import scala.util.Properties +import scala.collection.JavaConverters._ +import scala.collection.mutable.Stack + +import sbt._ +import sbt.Classpaths.publishTask +import sbt.Keys._ +import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion +import com.etsy.sbt.checkstyle.CheckstylePlugin.autoImport._ +import com.simplytyped.Antlr4Plugin._ +import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} +import com.typesafe.tools.mima.plugin.MimaKeys +import org.scalastyle.sbt.ScalastylePlugin.autoImport._ +import org.scalastyle.sbt.Tasks + +import spray.revolver.RevolverPlugin._ + +object BuildCommons { + + private val buildLocation = file(".").getAbsoluteFile.getParentFile + + val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, tokenProviderKafka010, sqlKafka010, avro) = Seq( + "catalyst", "sql", "hive", "hive-thriftserver", "token-provider-kafka-0-10", "sql-kafka-0-10", "avro" + ).map(ProjectRef(buildLocation, _)) + + val streamingProjects@Seq(streaming, streamingKafka010) = + Seq("streaming", "streaming-kafka-0-10").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 + + val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, + sparkGangliaLgpl, streamingKinesisAsl, + dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests) = + Seq("kubernetes", "mesos", "yarn", + "ganglia-lgpl", "streaming-kinesis-asl", + "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests").map(ProjectRef(buildLocation, _)) + + val assemblyProjects@Seq(networkYarn, streamingKafka010Assembly, streamingKinesisAslAssembly) = + Seq("network-yarn", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") + .map(ProjectRef(buildLocation, _)) + + val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples") + .map(ProjectRef(buildLocation, _)) + + val tools = ProjectRef(buildLocation, "tools") + // Root project. + val spark = ProjectRef(buildLocation, "spark") + val sparkHome = buildLocation + + val testTempDir = s"$sparkHome/target/tmp" + + val javaVersion = settingKey[String]("source and target JVM version for javac and scalac") +} + +object SparkBuild extends PomBuild { + + import BuildCommons._ + import scala.collection.mutable.Map + + val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty + + override val profiles = { + val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") + .orElse(Properties.propOrNone("sbt.maven.profiles")) match { + case None => Seq("sbt") + case Some(v) => + v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq + } + + // TODO: revisit for Scala 2.13 support + /* + Option(System.getProperty("scala.version")) + .filter(_.startsWith("2.11")) + .foreach { versionString => + System.setProperty("scala-2.11", "true") + } + if (System.getProperty("scala-2.11") == "") { + // To activate scala-2.10 profile, replace empty property value to non-empty value + // in the same way as Maven which handles -Dname as -Dname=true before executes build process. + // see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082 + System.setProperty("scala-2.11", "true") + } + */ + profiles + } + + Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { + case Some(v) => + v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) + case _ => + } + + override val userPropertiesMap = System.getProperties.asScala.toMap + + lazy val MavenCompile = config("m2r") extend(Compile) + lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") + + lazy val sparkGenjavadocSettings: Seq[sbt.Def.Setting[_]] = Seq( + libraryDependencies += compilerPlugin( + "com.typesafe.genjavadoc" %% "genjavadoc-plugin" % unidocGenjavadocVersion.value cross CrossVersion.full), + scalacOptions ++= Seq( + "-P:genjavadoc:out=" + (target.value / "java"), + "-P:genjavadoc:strictVisibility=true" // hide package private types + ) + ) + + lazy val scalaStyleRules = Project("scalaStyleRules", file("scalastyle")) + .settings( + libraryDependencies += "org.scalastyle" %% "scalastyle" % "1.0.0" + ) + + lazy val scalaStyleOnCompile = taskKey[Unit]("scalaStyleOnCompile") + + lazy val scalaStyleOnTest = taskKey[Unit]("scalaStyleOnTest") + + // We special case the 'println' lint rule to only be a warning on compile, because adding + // printlns for debugging is a common use case and is easy to remember to remove. + val scalaStyleOnCompileConfig: String = { + val in = "scalastyle-config.xml" + val out = "scalastyle-on-compile.generated.xml" + val replacements = Map( + """customId="println" level="error"""" -> """customId="println" level="warn"""" + ) + var contents = Source.fromFile(in).getLines.mkString("\n") + for ((k, v) <- replacements) { + require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.") + contents = contents.replace(k, v) + } + new PrintWriter(out) { + write(contents) + close() + } + out + } + + // Return a cached scalastyle task for a given configuration (usually Compile or Test) + private def cachedScalaStyle(config: Configuration) = Def.task { + val logger = streams.value.log + // We need a different cache dir per Configuration, otherwise they collide + val cacheDir = target.value / s"scalastyle-cache-${config.name}" + val cachedFun = FileFunction.cached(cacheDir, FilesInfo.lastModified, FilesInfo.exists) { + (inFiles: Set[File]) => { + val args: Seq[String] = Seq.empty + val scalaSourceV = Seq(file(scalaSource.in(config).value.getAbsolutePath)) + val configV = (baseDirectory in ThisBuild).value / scalaStyleOnCompileConfig + val configUrlV = scalastyleConfigUrl.in(config).value + val streamsV = streams.in(config).value + val failOnErrorV = true + val failOnWarningV = false + val scalastyleTargetV = scalastyleTarget.in(config).value + val configRefreshHoursV = scalastyleConfigRefreshHours.in(config).value + val targetV = target.in(config).value + val configCacheFileV = scalastyleConfigUrlCacheFile.in(config).value + + logger.info(s"Running scalastyle on ${name.value} in ${config.name}") + Tasks.doScalastyle(args, configV, configUrlV, failOnErrorV, failOnWarningV, scalaSourceV, + scalastyleTargetV, streamsV, configRefreshHoursV, targetV, configCacheFileV) + + Set.empty + } + } + + cachedFun(findFiles(scalaSource.in(config).value)) + } + + private def findFiles(file: File): Set[File] = if (file.isDirectory) { + file.listFiles().toSet.flatMap(findFiles) + file + } else { + Set(file) + } + + def enableScalaStyle: Seq[sbt.Def.Setting[_]] = Seq( + scalaStyleOnCompile := cachedScalaStyle(Compile).value, + scalaStyleOnTest := cachedScalaStyle(Test).value, + logLevel in scalaStyleOnCompile := Level.Warn, + logLevel in scalaStyleOnTest := Level.Warn, + (compile in Compile) := { + scalaStyleOnCompile.value + (compile in Compile).value + }, + (compile in Test) := { + scalaStyleOnTest.value + (compile in Test).value + } + ) + + lazy val sharedSettings = sparkGenjavadocSettings ++ + (if (sys.env.contains("NOLINT_ON_COMPILE")) Nil else enableScalaStyle) ++ Seq( + exportJars in Compile := true, + exportJars in Test := false, + javaHome := sys.env.get("JAVA_HOME") + .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) + .map(file), + incOptions := incOptions.value.withNameHashing(true), + publishMavenStyle := true, + unidocGenjavadocVersion := "0.16", + + // 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/maven2/", + DefaultMavenRepository, + Resolver.mavenLocal, + Resolver.file("local", file(Path.userHome.absolutePath + "/.ivy2/local"))(Resolver.ivyStylePatterns) + ), + externalResolvers := resolvers.value, + otherResolvers := SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))).value, + publishLocalConfiguration in MavenCompile := + new PublishConfiguration(None, "dotM2", packagedArtifacts.value, Seq(), ivyLoggingLevel.value), + publishMavenStyle in MavenCompile := true, + publishLocal in MavenCompile := publishTask(publishLocalConfiguration in MavenCompile, deliverLocal).value, + publishLocalBoth := Seq(publishLocal in MavenCompile, publishLocal).dependOn.value, + + javacOptions in (Compile, doc) ++= { + val versionParts = System.getProperty("java.version").split("[+.\\-]+", 3) + var major = versionParts(0).toInt + if (major == 1) major = versionParts(1).toInt + if (major >= 8) Seq("-Xdoclint:all", "-Xdoclint:-missing") else Seq.empty + }, + + javaVersion := SbtPomKeys.effectivePom.value.getProperties.get("java.version").asInstanceOf[String], + + javacOptions in Compile ++= Seq( + "-encoding", UTF_8.name(), + "-source", javaVersion.value + ), + // This -target and Xlint:unchecked options cannot be set in the Compile configuration scope since + // `javadoc` doesn't play nicely with them; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629 + // for additional discussion and explanation. + javacOptions in (Compile, compile) ++= Seq( + "-target", javaVersion.value, + "-Xlint:unchecked" + ), + + scalacOptions in Compile ++= Seq( + s"-target:jvm-${javaVersion.value}", + "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath // Required for relative source links in scaladoc + ), + + // Remove certain packages from Scaladoc + scalacOptions in (Compile, doc) := Seq( + "-groups", + "-skip-packages", Seq( + "org.apache.spark.api.python", + "org.apache.spark.network", + "org.apache.spark.deploy", + "org.apache.spark.util.collection" + ).mkString(":"), + "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc" + ) ++ { + // Do not attempt to scaladoc javadoc comments under 2.12 since it can't handle inner classes + if (scalaBinaryVersion.value == "2.12") Seq("-no-java-comments") else Seq.empty + }, + + // Implements -Xfatal-warnings, ignoring deprecation warnings. + // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410. + compile in Compile := { + val analysis = (compile in Compile).value + val out = streams.value + + def logProblem(l: (=> String) => Unit, f: File, p: xsbti.Problem) = { + l(f.toString + ":" + p.position.line.fold("")(_ + ":") + " " + p.message) + l(p.position.lineContent) + l("") + } + + var failed = 0 + analysis.infos.allInfos.foreach { case (k, i) => + i.reportedProblems foreach { p => + val deprecation = p.message.contains("is deprecated") + + if (!deprecation) { + failed = failed + 1 + } + + val printer: (=> String) => Unit = s => if (deprecation) { + out.log.warn(s) + } else { + out.log.error("[warn] " + s) + } + + logProblem(printer, k, p) + + } + } + + if (failed > 0) { + sys.error(s"$failed fatal warnings") + } + analysis + } + ) + + def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = { + val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]()) + projectsMap += (projectRef.project -> (existingSettings ++ settings)) + } + + // Note ordering of these settings matter. + /* Enable shared settings on all projects */ + (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ copyJarsProjects ++ Seq(spark, tools)) + .foreach(enable(sharedSettings ++ DependencyOverrides.settings ++ + ExcludedDependencies.settings ++ Checkstyle.settings)) + + /* Enable tests settings for all projects except examples, assembly and tools */ + (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) + + val mimaProjects = allProjects.filterNot { x => + Seq( + spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, + unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro + ).contains(x) + } + + mimaProjects.foreach { x => + enable(MimaBuild.mimaSettings(sparkHome, x))(x) + } + + /* Generate and pick the spark build info from extra-resources */ + enable(Core.settings)(core) + + /* Unsafe settings */ + enable(Unsafe.settings)(unsafe) + + /* + * Set up tasks to copy dependencies during packaging. This step can be disabled in the command + * line, so that dev/mima can run without trying to copy these files again and potentially + * causing issues. + */ + if (!"false".equals(System.getProperty("copyDependencies"))) { + copyJarsProjects.foreach(enable(CopyDependencies.settings)) + } + + /* Enable Assembly for all assembly projects */ + assemblyProjects.foreach(enable(Assembly.settings)) + + /* Package pyspark artifacts in a separate zip file for YARN. */ + enable(PySparkAssembly.settings)(assembly) + + /* Enable unidoc only for the root spark project */ + enable(Unidoc.settings)(spark) + + /* Catalyst ANTLR generation settings */ + enable(Catalyst.settings)(catalyst) + + /* Spark SQL Core console settings */ + enable(SQL.settings)(sql) + + /* Hive console settings */ + enable(Hive.settings)(hive) + + // SPARK-14738 - Remove docker tests from main Spark build + // enable(DockerIntegrationTests.settings)(dockerIntegrationTests) + + enable(KubernetesIntegrationTests.settings)(kubernetesIntegrationTests) + + /** + * Adds the ability to run the spark shell directly from SBT without building an assembly + * jar. + * + * Usage: `build/sbt sparkShell` + */ + val sparkShell = taskKey[Unit]("start a spark-shell.") + val sparkPackage = inputKey[Unit]( + s""" + |Download and run a spark package. + |Usage `builds/sbt "sparkPackage [args] + """.stripMargin) + val sparkSql = taskKey[Unit]("starts the spark sql CLI.") + + enable(Seq( + connectInput in run := true, + fork := true, + outputStrategy in run := Some (StdoutOutput), + + javaOptions += "-Xmx2g", + + sparkShell := { + (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value + }, + + sparkPackage := { + import complete.DefaultParsers._ + val packages :: className :: otherArgs = spaceDelimited(" [args]").parsed.toList + val scalaRun = (runner in run).value + val classpath = (fullClasspath in Runtime).value + val args = Seq("--packages", packages, "--class", className, (Keys.`package` in Compile in LocalProject("core")) + .value.getCanonicalPath) ++ otherArgs + println(args) + scalaRun.run("org.apache.spark.deploy.SparkSubmit", classpath.map(_.data), args, streams.value.log) + }, + + javaOptions in Compile += "-Dspark.master=local", + + sparkSql := { + (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value + } + ))(assembly) + + enable(Seq(sparkShell := sparkShell in LocalProject("assembly")))(spark) + + // TODO: move this to its upstream project. + override def projectDefinitions(baseDirectory: File): Seq[Project] = { + super.projectDefinitions(baseDirectory).map { x => + if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*) + else x.settings(Seq[Setting[_]](): _*) + } ++ Seq[Project](OldDeps.project) + } + + if (!sys.env.contains("SERIAL_SBT_TESTS")) { + allProjects.foreach(enable(SparkParallelTestGrouping.settings)) + } +} + +object SparkParallelTestGrouping { + // Settings for parallelizing tests. The basic strategy here is to run the slowest suites (or + // collections of suites) in their own forked JVMs, allowing us to gain parallelism within a + // SBT project. Here, we take a whitelisting approach where the default behavior is to run all + // tests sequentially in a single JVM, requiring us to manually opt-in to the extra parallelism. + // + // There are a reasons why such a whitelist approach is good: + // + // 1. Launching one JVM per suite adds significant overhead for short-running suites. In + // addition to JVM startup time and JIT warmup, it appears that initialization of Derby + // metastores can be very slow so creating a fresh warehouse per suite is inefficient. + // + // 2. When parallelizing within a project we need to give each forked JVM a different tmpdir + // so that the metastore warehouses do not collide. Unfortunately, it seems that there are + // some tests which have an overly tight dependency on the default tmpdir, so those fragile + // tests need to continue re-running in the default configuration (or need to be rewritten). + // Fixing that problem would be a huge amount of work for limited payoff in most cases + // because most test suites are short-running. + // + + private val testsWhichShouldRunInTheirOwnDedicatedJvm = Set( + "org.apache.spark.DistributedSuite", + "org.apache.spark.sql.catalyst.expressions.DateExpressionsSuite", + "org.apache.spark.sql.catalyst.expressions.HashExpressionsSuite", + "org.apache.spark.sql.catalyst.expressions.CastSuite", + "org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite", + "org.apache.spark.sql.hive.HiveExternalCatalogSuite", + "org.apache.spark.sql.hive.StatisticsSuite", + "org.apache.spark.sql.hive.execution.HiveCompatibilitySuite", + "org.apache.spark.sql.hive.client.VersionsSuite", + "org.apache.spark.sql.hive.client.HiveClientVersions", + "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite", + "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.SparkSQLEnvSuite", + "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", + "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", + "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", + "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" + ) + + private val DEFAULT_TEST_GROUP = "default_test_group" + + private def testNameToTestGroup(name: String): String = name match { + case _ if testsWhichShouldRunInTheirOwnDedicatedJvm.contains(name) => name + case _ => DEFAULT_TEST_GROUP + } + + lazy val settings = Seq( + testGrouping in Test := { + val tests: Seq[TestDefinition] = (definedTests in Test).value + val defaultForkOptions = ForkOptions( + bootJars = Nil, + javaHome = javaHome.value, + connectInput = connectInput.value, + outputStrategy = outputStrategy.value, + runJVMOptions = (javaOptions in Test).value, + workingDirectory = Some(baseDirectory.value), + envVars = (envVars in Test).value + ) + tests.groupBy(test => testNameToTestGroup(test.name)).map { case (groupName, groupTests) => + val forkOptions = { + if (groupName == DEFAULT_TEST_GROUP) { + defaultForkOptions + } else { + defaultForkOptions.copy(runJVMOptions = defaultForkOptions.runJVMOptions ++ + Seq(s"-Djava.io.tmpdir=${baseDirectory.value}/target/tmp/$groupName")) + } + } + new Tests.Group( + name = groupName, + tests = groupTests, + runPolicy = Tests.SubProcess(forkOptions)) + } + }.toSeq + ) +} + +object Core { + lazy val settings = Seq( + resourceGenerators in Compile += Def.task { + val buildScript = baseDirectory.value + "/../build/spark-build-info" + val targetDir = baseDirectory.value + "/target/extra-resources/" + val command = Seq("bash", buildScript, targetDir, version.value) + Process(command).!! + val propsFile = baseDirectory.value / "target" / "extra-resources" / "spark-version-info.properties" + Seq(propsFile) + }.taskValue + ) +} + +object Unsafe { + lazy val settings = Seq( + // This option is needed to suppress warnings from sun.misc.Unsafe usage + javacOptions in Compile += "-XDignore.symbol.file" + ) +} + + +object DockerIntegrationTests { + // This serves to override the override specified in DependencyOverrides: + lazy val settings = Seq( + dependencyOverrides += "com.google.guava" % "guava" % "18.0", + resolvers += "DB2" at "https://app.camunda.com/nexus/content/repositories/public/", + libraryDependencies += "com.oracle" % "ojdbc6" % "11.2.0.1.0" from "https://app.camunda.com/nexus/content/repositories/public/com/oracle/ojdbc6/11.2.0.1.0/ojdbc6-11.2.0.1.0.jar" // scalastyle:ignore + ) +} + +/** + * These settings run a hardcoded configuration of the Kubernetes integration tests using + * minikube. Docker images will have the "dev" tag, and will be overwritten every time the + * integration tests are run. The integration tests are actually bound to the "test" phase, + * so running "test" on this module will run the integration tests. + * + * There are two ways to run the tests: + * - the "tests" task builds docker images and runs the test, so it's a little slow. + * - the "run-its" task just runs the tests on a pre-built set of images. + * + * Note that this does not use the shell scripts that the maven build uses, which are more + * configurable. This is meant as a quick way for developers to run these tests against their + * local changes. + */ +object KubernetesIntegrationTests { + import BuildCommons._ + + val dockerBuild = TaskKey[Unit]("docker-imgs", "Build the docker images for ITs.") + val runITs = TaskKey[Unit]("run-its", "Only run ITs, skip image build.") + val imageTag = settingKey[String]("Tag to use for images built during the test.") + val namespace = settingKey[String]("Namespace where to run pods.") + + // Hack: this variable is used to control whether to build docker images. It's updated by + // the tasks below in a non-obvious way, so that you get the functionality described in + // the scaladoc above. + private var shouldBuildImage = true + + lazy val settings = Seq( + imageTag := "dev", + namespace := "default", + dockerBuild := { + if (shouldBuildImage) { + val dockerTool = s"$sparkHome/bin/docker-image-tool.sh" + val bindingsDir = s"$sparkHome/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings" + val cmd = Seq(dockerTool, "-m", + "-t", imageTag.value, + "-p", s"$bindingsDir/python/Dockerfile", + "-R", s"$bindingsDir/R/Dockerfile", + "build" + ) + val ec = Process(cmd).! + if (ec != 0) { + throw new IllegalStateException(s"Process '${cmd.mkString(" ")}' exited with $ec.") + } + } + shouldBuildImage = true + }, + runITs := Def.taskDyn { + shouldBuildImage = false + Def.task { + (test in Test).value + } + }.value, + test in Test := (test in Test).dependsOn(dockerBuild).value, + javaOptions in Test ++= Seq( + "-Dspark.kubernetes.test.deployMode=minikube", + s"-Dspark.kubernetes.test.imageTag=${imageTag.value}", + s"-Dspark.kubernetes.test.namespace=${namespace.value}", + s"-Dspark.kubernetes.test.unpackSparkDir=$sparkHome" + ), + // Force packaging before building images, so that the latest code is tested. + dockerBuild := dockerBuild.dependsOn(packageBin in Compile in assembly) + .dependsOn(packageBin in Compile in examples).value + ) +} + +/** + * Overrides to work around sbt's dependency resolution being different from Maven's. + */ +object DependencyOverrides { + lazy val settings = Seq( + dependencyOverrides += "com.google.guava" % "guava" % "14.0.1", + dependencyOverrides += "xerces" % "xercesImpl" % "2.12.0", + dependencyOverrides += "jline" % "jline" % "2.14.6", + dependencyOverrides += "org.apache.avro" % "avro" % "1.8.2") +} + +/** + * This excludes library dependencies in sbt, which are specified in maven but are + * not needed by sbt build. + */ +object ExcludedDependencies { + lazy val settings = Seq( + libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") } + ) +} + +/** + * Project to pull previous artifacts of Spark for generating Mima excludes. + */ +object OldDeps { + + lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings) + + lazy val allPreviousArtifactKeys = Def.settingDyn[Seq[Set[ModuleID]]] { + SparkBuild.mimaProjects + .map { project => MimaKeys.mimaPreviousArtifacts in project } + .map(k => Def.setting(k.value)) + .join + } + + def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq( + name := "old-deps", + libraryDependencies := allPreviousArtifactKeys.value.flatten + ) +} + +object Catalyst { + lazy val settings = antlr4Settings ++ Seq( + antlr4Version in Antlr4 := SbtPomKeys.effectivePom.value.getProperties.get("antlr4.version").asInstanceOf[String], + antlr4PackageName in Antlr4 := Some("org.apache.spark.sql.catalyst.parser"), + antlr4GenListener in Antlr4 := true, + antlr4GenVisitor in Antlr4 := true, + antlr4TreatWarningsAsErrors in Antlr4 := true + ) +} + +object SQL { + lazy val settings = Seq( + initialCommands in console := + """ + |import org.apache.spark.SparkContext + |import org.apache.spark.sql.SQLContext + |import org.apache.spark.sql.catalyst.analysis._ + |import org.apache.spark.sql.catalyst.dsl._ + |import org.apache.spark.sql.catalyst.errors._ + |import org.apache.spark.sql.catalyst.expressions._ + |import org.apache.spark.sql.catalyst.plans.logical._ + |import org.apache.spark.sql.catalyst.rules._ + |import org.apache.spark.sql.catalyst.util._ + |import org.apache.spark.sql.execution + |import org.apache.spark.sql.functions._ + |import org.apache.spark.sql.types._ + | + |val sc = new SparkContext("local[*]", "dev-shell") + |val sqlContext = new SQLContext(sc) + |import sqlContext.implicits._ + |import sqlContext._ + """.stripMargin, + cleanupCommands in console := "sc.stop()" + ) +} + +object Hive { + + lazy val settings = Seq( + // Specially disable assertions since some Hive tests fail them + javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"), + // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings + // only for this subproject. + scalacOptions := (scalacOptions map { currentOpts: Seq[String] => + currentOpts.filterNot(_ == "-deprecation") + }).value, + initialCommands in console := + """ + |import org.apache.spark.SparkContext + |import org.apache.spark.sql.catalyst.analysis._ + |import org.apache.spark.sql.catalyst.dsl._ + |import org.apache.spark.sql.catalyst.errors._ + |import org.apache.spark.sql.catalyst.expressions._ + |import org.apache.spark.sql.catalyst.plans.logical._ + |import org.apache.spark.sql.catalyst.rules._ + |import org.apache.spark.sql.catalyst.util._ + |import org.apache.spark.sql.execution + |import org.apache.spark.sql.functions._ + |import org.apache.spark.sql.hive._ + |import org.apache.spark.sql.hive.test.TestHive._ + |import org.apache.spark.sql.hive.test.TestHive.implicits._ + |import org.apache.spark.sql.types._""".stripMargin, + cleanupCommands in console := "sparkContext.stop()", + // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce + // in order to generate golden files. This is only required for developers who are adding new + // new query tests. + fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") } + ) +} + +object Assembly { + import sbtassembly.AssemblyUtils._ + import sbtassembly.Plugin._ + import AssemblyKeys._ + + val hadoopVersion = taskKey[String]("The version of hadoop that spark is compiled against.") + + lazy val settings = assemblySettings ++ Seq( + test in assembly := {}, + hadoopVersion := { + sys.props.get("hadoop.version") + .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) + }, + jarName in assembly := { + if (moduleName.value.contains("streaming-kafka-0-10-assembly") + || moduleName.value.contains("streaming-kinesis-asl-assembly")) { + s"${moduleName.value}-${version.value}.jar" + } else { + s"${moduleName.value}-${version.value}-hadoop${hadoopVersion.value}.jar" + } + }, + jarName in (Test, assembly) := s"${moduleName.value}-test-${version.value}.jar", + mergeStrategy in assembly := { + case m if m.toLowerCase(Locale.ROOT).endsWith("manifest.mf") + => MergeStrategy.discard + case m if m.toLowerCase(Locale.ROOT).matches("meta-inf.*\\.sf$") + => MergeStrategy.discard + case "log4j.properties" => MergeStrategy.discard + case m if m.toLowerCase(Locale.ROOT).startsWith("meta-inf/services/") + => MergeStrategy.filterDistinctLines + case "reference.conf" => MergeStrategy.concat + case _ => MergeStrategy.first + } + ) +} + +object PySparkAssembly { + import sbtassembly.Plugin._ + import AssemblyKeys._ + import java.util.zip.{ZipOutputStream, ZipEntry} + + lazy val settings = Seq( + // Use a resource generator to copy all .py files from python/pyspark into a managed directory + // to be included in the assembly. We can't just add "python/" to the assembly's resource dir + // list since that will copy unneeded / unwanted files. + resourceGenerators in Compile += Def.macroValueI(resourceManaged in Compile map { outDir: File => + val src = new File(BuildCommons.sparkHome, "python/pyspark") + val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip") + zipFile.delete() + zipRecursive(src, zipFile) + Seq.empty[File] + }).value + ) + + private def zipRecursive(source: File, destZipFile: File) = { + val destOutput = new ZipOutputStream(new FileOutputStream(destZipFile)) + addFilesToZipStream("", source, destOutput) + destOutput.flush() + destOutput.close() + } + + private def addFilesToZipStream(parent: String, source: File, output: ZipOutputStream): Unit = { + if (source.isDirectory()) { + output.putNextEntry(new ZipEntry(parent + source.getName())) + for (file <- source.listFiles()) { + addFilesToZipStream(parent + source.getName() + File.separator, file, output) + } + } else { + val in = new FileInputStream(source) + output.putNextEntry(new ZipEntry(parent + source.getName())) + val buf = new Array[Byte](8192) + var n = 0 + while (n != -1) { + n = in.read(buf) + if (n != -1) { + output.write(buf, 0, n) + } + } + output.closeEntry() + in.close() + } + } + +} + +object Unidoc { + + import BuildCommons._ + import sbtunidoc.Plugin._ + import UnidocKeys._ + + private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = { + packages + .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(_.getCanonicalPath.contains("org/apache/spark/rpc"))) + .map(_.filterNot(f => + 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(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"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution"))) + .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] = { + classpaths + .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka-clients-0\.10.*"""))) + .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka_2\..*-0\.10.*"""))) + } + + val unidocSourceBase = settingKey[String]("Base URL of source links in Scaladoc.") + + lazy val settings = scalaJavaUnidocSettings ++ Seq ( + publish := {}, + + unidocProjectFilter in(ScalaUnidoc, unidoc) := + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, + yarn, tags, streamingKafka010, sqlKafka010, avro), + unidocProjectFilter in(JavaUnidoc, unidoc) := + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, + yarn, tags, streamingKafka010, sqlKafka010, avro), + + unidocAllClasspaths in (ScalaUnidoc, unidoc) := { + ignoreClasspaths((unidocAllClasspaths in (ScalaUnidoc, unidoc)).value) + }, + + unidocAllClasspaths in (JavaUnidoc, unidoc) := { + ignoreClasspaths((unidocAllClasspaths in (JavaUnidoc, unidoc)).value) + }, + + // Skip actual catalyst, but include the subproject. + // Catalyst is not public API and contains quasiquotes which break scaladoc. + unidocAllSources in (ScalaUnidoc, unidoc) := { + ignoreUndocumentedPackages((unidocAllSources in (ScalaUnidoc, unidoc)).value) + }, + + // Skip class names containing $ and some internal packages in Javadocs + unidocAllSources in (JavaUnidoc, unidoc) := { + ignoreUndocumentedPackages((unidocAllSources in (JavaUnidoc, unidoc)).value) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop"))) + }, + + javacOptions in (JavaUnidoc, unidoc) := Seq( + "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", + "-public", + "-noqualifier", "java.lang", + "-tag", """example:a:Example\:""", + "-tag", """note:a:Note\:""", + "-tag", "group:X", + "-tag", "tparam:X", + "-tag", "constructor:X", + "-tag", "todo:X", + "-tag", "groupname:X" + ), + + // Use GitHub repository for Scaladoc source links + unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}", + + scalacOptions in (ScalaUnidoc, unidoc) ++= Seq( + "-groups", // Group similar methods together based on the @group annotation. + "-skip-packages", "org.apache.hadoop", + "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath + ) ++ ( + // Add links to sources when generating Scaladoc for a non-snapshot release + if (!isSnapshot.value) { + Opts.doc.sourceUrl(unidocSourceBase.value + "€{FILE_PATH}.scala") + } else { + Seq() + } + ) + ) +} + +object Checkstyle { + lazy val settings = Seq( + checkstyleSeverityLevel := Some(CheckstyleSeverityLevel.Error), + javaSource in (Compile, checkstyle) := baseDirectory.value / "src/main/java", + javaSource in (Test, checkstyle) := baseDirectory.value / "src/test/java", + checkstyleConfigLocation := CheckstyleConfigLocation.File("dev/checkstyle.xml"), + checkstyleOutputFile := baseDirectory.value / "target/checkstyle-output.xml", + checkstyleOutputFile in Test := baseDirectory.value / "target/checkstyle-output.xml" + ) +} + +object CopyDependencies { + + val copyDeps = TaskKey[Unit]("copyDeps", "Copies needed dependencies to the build directory.") + val destPath = (crossTarget in Compile) { _ / "jars"} + + lazy val settings = Seq( + copyDeps := { + val dest = destPath.value + if (!dest.isDirectory() && !dest.mkdirs()) { + throw new IOException("Failed to create jars directory.") + } + + (dependencyClasspath in Compile).value.map(_.data) + .filter { jar => jar.isFile() } + .foreach { jar => + val destJar = new File(dest, jar.getName()) + if (destJar.isFile()) { + destJar.delete() + } + Files.copy(jar.toPath(), destJar.toPath()) + } + }, + crossTarget in (Compile, packageBin) := destPath.value, + packageBin in Compile := (packageBin in Compile).dependsOn(copyDeps).value + ) + +} + +object TestSettings { + import BuildCommons._ + + // TODO revisit for Scala 2.13 support + private val scalaBinaryVersion = "2.12" + /* + if (System.getProperty("scala-2.11") == "true") { + "2.11" + } else { + "2.12" + } + */ + + private val defaultExcludedTags = Seq("org.apache.spark.tags.ChromeUITest") + + lazy val settings = Seq ( + // Fork new JVMs for tests and set Java options for those + fork := true, + // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes + // launched by the tests have access to the correct test-time classpath. + envVars in Test ++= Map( + "SPARK_DIST_CLASSPATH" -> + (fullClasspath in Test).value.files.map(_.getAbsolutePath) + .mkString(File.pathSeparator).stripSuffix(File.pathSeparator), + "SPARK_PREPEND_CLASSES" -> "1", + "SPARK_SCALA_VERSION" -> scalaBinaryVersion, + "SPARK_TESTING" -> "1", + "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), + javaOptions in Test += s"-Djava.io.tmpdir=$testTempDir", + javaOptions in Test += "-Dspark.test.home=" + sparkHome, + javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test += "-Dspark.port.maxRetries=100", + javaOptions in Test += "-Dspark.master.rest.enabled=false", + javaOptions in Test += "-Dspark.memory.debugFill=true", + javaOptions in Test += "-Dspark.ui.enabled=false", + javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", + javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", + javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=false", + javaOptions in Test += "-Dderby.system.durability=test", + javaOptions in Test += "-Dio.netty.tryReflectionSetAccessible=true", + javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) + .map { case (k,v) => s"-D$k=$v" }.toSeq, + javaOptions in Test += "-ea", + // SPARK-29282 This is for consistency between JDK8 and JDK11. + javaOptions in Test ++= "-Xmx4g -Xss4m -XX:+UseParallelGC -XX:-UseDynamicNumberOfGCThreads" + .split(" ").toSeq, + javaOptions += "-Xmx3g", + // Exclude tags defined in a system property + testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, + sys.props.get("test.exclude.tags").map { tags => + tags.split(",").flatMap { tag => Seq("-l", tag) }.toSeq + }.getOrElse(Nil): _*), + testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, + sys.props.get("test.default.exclude.tags").map(tags => tags.split(",").toSeq) + .map(tags => tags.filter(!_.trim.isEmpty)).getOrElse(defaultExcludedTags) + .flatMap(tag => Seq("-l", tag)): _*), + testOptions in Test += Tests.Argument(TestFrameworks.JUnit, + sys.props.get("test.exclude.tags").map { tags => + Seq("--exclude-categories=" + tags) + }.getOrElse(Nil): _*), + // Show full stack trace and duration in test cases. + testOptions in Test += Tests.Argument("-oDF"), + testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), + // Enable Junit testing. + libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test", + // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project + // can run in parallel with one another. It does NOT control whether tests execute in parallel + // within the same JVM (which is controlled by `testForkedParallel`) or whether test cases + // within the same suite can run in parallel (which is a ScalaTest runner option which is passed + // to the underlying runner but is not a SBT-level configuration). This needs to be `true` in + // order for the extra parallelism enabled by `SparkParallelTestGrouping` to take effect. + // The `SERIAL_SBT_TESTS` check is here so the extra parallelism can be feature-flagged. + parallelExecution in Test := { if (sys.env.contains("SERIAL_SBT_TESTS")) false else true }, + // Make sure the test temp directory exists. + resourceGenerators in Test += Def.macroValueI(resourceManaged in Test map { outDir: File => + var dir = new File(testTempDir) + if (!dir.isDirectory()) { + // Because File.mkdirs() can fail if multiple callers are trying to create the same + // parent directory, this code tries to create parents one at a time, and avoids + // failures when the directories have been created by somebody else. + val stack = new Stack[File]() + while (!dir.isDirectory()) { + stack.push(dir) + dir = dir.getParentFile() + } + + while (stack.nonEmpty) { + val d = stack.pop() + require(d.mkdir() || d.isDirectory(), s"Failed to create directory $d") + } + } + Seq.empty[File] + }).value, + concurrentRestrictions in Global := { + // The number of concurrent test groups is empirically chosen based on experience + // with Jenkins flakiness. + if (sys.env.contains("SERIAL_SBT_TESTS")) (concurrentRestrictions in Global).value + else Seq(Tags.limit(Tags.ForkedTestGroup, 4)) + } + ) + +} diff --git a/project/build.properties b/project/build.properties new file mode 100644 index 000000000000..23aa187fb35a --- /dev/null +++ b/project/build.properties @@ -0,0 +1,17 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +sbt.version=0.13.18 diff --git a/project/plugins.sbt b/project/plugins.sbt new file mode 100644 index 000000000000..5f21d8126e48 --- /dev/null +++ b/project/plugins.sbt @@ -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. + */ + +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.25" + +// checkstyle uses guava 23.0. +libraryDependencies += "com.google.guava" % "guava" % "23.0" + +// need to make changes to uptake sbt 1.0 support in "com.eed3si9n" % "sbt-assembly" % "1.14.5" +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") + +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "5.2.4") + +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 +addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") + +// need to make changes to uptake sbt 1.0 support in "com.eed3si9n" % "sbt-unidoc" % "0.4.1" +addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.3") + +// need to make changes to uptake sbt 1.0 support in "com.cavorite" % "sbt-avro-1-7" % "1.1.2" +addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") + +addSbtPlugin("io.spray" % "sbt-revolver" % "0.9.1") + +libraryDependencies += "org.ow2.asm" % "asm" % "7.2" + +libraryDependencies += "org.ow2.asm" % "asm-commons" % "7.2" + +// sbt 1.0.0 support: https://github.com/ihji/sbt-antlr4/issues/14 +addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.13") + +// Spark uses a custom fork of the sbt-pom-reader plugin which contains a patch to fix issues +// related to test-jar dependencies (https://github.com/sbt/sbt-pom-reader/pull/14). The source for +// this fork is published at https://github.com/JoshRosen/sbt-pom-reader/tree/v1.0.0-spark +// and corresponds to commit b160317fcb0b9d1009635a7c5aa05d0f3be61936 in that repository. +// In the long run, we should try to merge our patch upstream and switch to an upstream version of +// the plugin; this is tracked at SPARK-14401. + +addSbtPlugin("org.spark-project" % "sbt-pom-reader" % "1.0.0-spark") From 6d56a8300d3e4f9f06917363813ac90aa137fa33 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 12 Jun 2020 11:59:40 +0800 Subject: [PATCH 60/75] fix error cases --- .../org/apache/spark/sql/hive/execution/HiveComparisonTest.scala | 1 - 1 file changed, 1 deletion(-) 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 2f0f88260641..2e4c01830432 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 @@ -315,7 +315,6 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" - print(cachedAnswerName) new File(answerCache, cachedAnswerName) } From 5716f40f8233b8cf8be2940ffe8132aa92e61718 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 12 Jun 2020 12:39:35 +0800 Subject: [PATCH 61/75] remove cmd test --- core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 39da57dd954a..b0f68880f1d8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -476,7 +476,7 @@ - + From 64acff3f4e8c6874796ab4cbdd9349b490cd1c47 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Fri, 12 Jun 2020 16:30:35 +0800 Subject: [PATCH 62/75] 'empty' From d26bd34c11aa66303e02b66a6335429ee327420e Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 00:40:56 +0800 Subject: [PATCH 63/75] fix test case error --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 6 ++++++ ...f845cf4 => decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972} | 0 .../golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 | 1 - ...288dde => decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b} | 0 ...354aba => decimal_1-11-f2f975b73220512d4bf2b9bd93354aba} | 0 ...8565f8 => decimal_1-12-2c2325880ea79c8e308398d46c8565f8} | 0 ...3092d3 => decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3} | 0 ...fc52f0 => decimal_1-14-e45935cfffb9045394e804d0d1fc52f0} | 0 ...ad0972 => decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4} | 0 ...97cfa707 => decimal_1-2-ee665100ca5de3a006df43e97cfa707} | 0 ...30a6653 => decimal_1-3-80fc87cab17ceffea334afbb230a6653} | 0 ...1afe119 => decimal_1-4-5dd925bba25f735bfd6442a841afe119} | 0 ...dfeeb256 => decimal_1-5-bfab296ca5693e647e33899dfeeb256} | 0 ...dc367a5 => decimal_1-6-a402201ed5159941384d40e09dc367a5} | 0 ...23b1cc5b => decimal_1-7-2cfd7d00bc37a8e433ad005896173c1} | 0 ...96173c1 => decimal_1-8-84cd75e494d113a48c4145298177d6d8} | 0 ...177d6d8 => decimal_1-9-e4e90927ac59f5920de3dc61c3288dde} | 0 .../ql/src/test/queries/clientpositive/decimal_1.q | 1 - sql/hive/src/test/resources/spark-version-info.properties | 6 ++++++ 19 files changed, 12 insertions(+), 2 deletions(-) rename sql/hive/src/test/resources/golden/{decimal_1-16-31ecaab3afa056fcc656d6e54f845cf4 => decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972} (100%) delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 rename sql/hive/src/test/resources/golden/{decimal_1-10-e4e90927ac59f5920de3dc61c3288dde => decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-12-f2f975b73220512d4bf2b9bd93354aba => decimal_1-11-f2f975b73220512d4bf2b9bd93354aba} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-13-2c2325880ea79c8e308398d46c8565f8 => decimal_1-12-2c2325880ea79c8e308398d46c8565f8} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-14-c4c33bdb9f3c6cad77552f0f353092d3 => decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-15-e45935cfffb9045394e804d0d1fc52f0 => decimal_1-14-e45935cfffb9045394e804d0d1fc52f0} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972 => decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-3-ee665100ca5de3a006df43e97cfa707 => decimal_1-2-ee665100ca5de3a006df43e97cfa707} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-4-80fc87cab17ceffea334afbb230a6653 => decimal_1-3-80fc87cab17ceffea334afbb230a6653} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-5-5dd925bba25f735bfd6442a841afe119 => decimal_1-4-5dd925bba25f735bfd6442a841afe119} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-6-bfab296ca5693e647e33899dfeeb256 => decimal_1-5-bfab296ca5693e647e33899dfeeb256} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-7-a402201ed5159941384d40e09dc367a5 => decimal_1-6-a402201ed5159941384d40e09dc367a5} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-11-c20dea9d716bef1bdbdef71323b1cc5b => decimal_1-7-2cfd7d00bc37a8e433ad005896173c1} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-8-2cfd7d00bc37a8e433ad005896173c1 => decimal_1-8-84cd75e494d113a48c4145298177d6d8} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-9-84cd75e494d113a48c4145298177d6d8 => decimal_1-9-e4e90927ac59f5920de3dc61c3288dde} (100%) create mode 100644 sql/hive/src/test/resources/spark-version-info.properties 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 db1f6fbd97d9..82af7dceb27f 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 @@ -39,6 +39,8 @@ 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 originalLegacyAllowCastNumericToTimestamp = + TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -58,6 +60,8 @@ 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") + // Ensures that cast numeric to timestamp enabled so that we can test them + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) RuleExecutor.resetMetrics() } @@ -68,6 +72,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_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + originalLegacyAllowCastNumericToTimestamp) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) diff --git a/sql/hive/src/test/resources/golden/decimal_1-16-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-16-31ecaab3afa056fcc656d6e54f845cf4 rename to sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-1-dce645ca43d2bf888c4d75c456e42f3 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-e4e90927ac59f5920de3dc61c3288dde b/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-10-e4e90927ac59f5920de3dc61c3288dde rename to sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-f2f975b73220512d4bf2b9bd93354aba b/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-12-f2f975b73220512d4bf2b9bd93354aba rename to sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-2c2325880ea79c8e308398d46c8565f8 b/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-13-2c2325880ea79c8e308398d46c8565f8 rename to sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-c4c33bdb9f3c6cad77552f0f353092d3 b/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-14-c4c33bdb9f3c6cad77552f0f353092d3 rename to sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 diff --git a/sql/hive/src/test/resources/golden/decimal_1-15-e45935cfffb9045394e804d0d1fc52f0 b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-15-e45935cfffb9045394e804d0d1fc52f0 rename to sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972 b/sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-2-6742a91ba2b9fa9c906d30d4d0ad0972 rename to sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-ee665100ca5de3a006df43e97cfa707 b/sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-3-ee665100ca5de3a006df43e97cfa707 rename to sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-4-80fc87cab17ceffea334afbb230a6653 rename to sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-5dd925bba25f735bfd6442a841afe119 b/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-5-5dd925bba25f735bfd6442a841afe119 rename to sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-bfab296ca5693e647e33899dfeeb256 b/sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-6-bfab296ca5693e647e33899dfeeb256 rename to sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-a402201ed5159941384d40e09dc367a5 b/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-7-a402201ed5159941384d40e09dc367a5 rename to sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-c20dea9d716bef1bdbdef71323b1cc5b b/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-11-c20dea9d716bef1bdbdef71323b1cc5b rename to sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-2cfd7d00bc37a8e433ad005896173c1 b/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-8-2cfd7d00bc37a8e433ad005896173c1 rename to sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-84cd75e494d113a48c4145298177d6d8 b/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-9-84cd75e494d113a48c4145298177d6d8 rename to sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q index 7ccfde4bcdcf..f52b1923eb06 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q @@ -1,5 +1,4 @@ set hive.fetch.task.conversion=more; -set spark.sql.legacy.allowCastNumericToTimestamp=true; drop table if exists decimal_1; diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties new file mode 100644 index 000000000000..69db78bdf332 --- /dev/null +++ b/sql/hive/src/test/resources/spark-version-info.properties @@ -0,0 +1,6 @@ +version=3.1.0 +user=dcadmin +revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f +branch=31710-fix-compatibility +date=2020-06-03T13:50:55Z +url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From 9545c9f17fc49c8d73b1a8f2fe9854244e91d19d Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 00:44:22 +0800 Subject: [PATCH 64/75] delete test file --- sql/hive/src/test/resources/spark-version-info.properties | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 sql/hive/src/test/resources/spark-version-info.properties diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/sql/hive/src/test/resources/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From 01f42e61b079f52e4816de1c0ee0cb08b6dffd5c Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 11:27:44 +0800 Subject: [PATCH 65/75] fix python style --- python/pyspark/sql/dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 593316bf67fc..ec83beaa6d44 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -535,8 +535,8 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving >>> from pyspark.sql.functions import timestamp_seconds - >>> sdf.select('name', timestamp_seconds(sdf.time).alias("time")) - ... .withWatermark('time', '10 minutes') + >>> sdf.select('name', timestamp_seconds(sdf.time).alias('time')) \ + ... .withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From e29409d9f358082fe999e079bc2a38b32bcc89c9 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 11:28:44 +0800 Subject: [PATCH 66/75] fix python style --- python/pyspark/sql/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ec83beaa6d44..91c541ccfabe 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -536,7 +536,7 @@ def withWatermark(self, eventTime, delayThreshold): >>> from pyspark.sql.functions import timestamp_seconds >>> sdf.select('name', timestamp_seconds(sdf.time).alias('time')) \ - ... .withWatermark('time', '10 minutes') + ... .withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 183d4188b62943a1a54664f58bc55111b107e047 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 15:17:02 +0800 Subject: [PATCH 67/75] trigger rebuild From 1764c769a928fbaa35e9b90866cdd17bf7705b23 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 20:14:10 +0800 Subject: [PATCH 68/75] fix python style --- python/pyspark/sql/dataframe.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 91c541ccfabe..3ad899bcc367 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -535,8 +535,9 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving >>> from pyspark.sql.functions import timestamp_seconds - >>> sdf.select('name', timestamp_seconds(sdf.time).alias('time')) \ - ... .withWatermark('time', '10 minutes') + >>> sdf.select( + ... 'name', + ... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 74eb6caed6702331406633e9d201683d34894c04 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sat, 13 Jun 2020 20:33:29 +0800 Subject: [PATCH 69/75] improve python style --- python/pyspark/sql/dataframe.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 3ad899bcc367..cb057912241a 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -537,7 +537,8 @@ def withWatermark(self, eventTime, delayThreshold): >>> from pyspark.sql.functions import timestamp_seconds >>> sdf.select( ... 'name', - ... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes') + ... timestamp_seconds(sdf.time).alias('time')) + ... .withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 5f0671475788aaa268c3b5481e6dff1f96c7a723 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Sun, 14 Jun 2020 01:11:43 +0800 Subject: [PATCH 70/75] fix python style --- python/pyspark/sql/dataframe.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index cb057912241a..3ad899bcc367 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -537,8 +537,7 @@ def withWatermark(self, eventTime, delayThreshold): >>> from pyspark.sql.functions import timestamp_seconds >>> sdf.select( ... 'name', - ... timestamp_seconds(sdf.time).alias('time')) - ... .withWatermark('time', '10 minutes') + ... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: From 8fe1960ef3a0c598a626b7024820b74cec787642 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Mon, 15 Jun 2020 12:08:06 +0800 Subject: [PATCH 71/75] improve test queality --- ...ull testing-0-6a01a94ef1b0d29152c88cd3083fd70b} | 0 ... null testing-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...tamp cast #3-0-89ef5480d0dd001cec4642de58f8632f | 1 + ...stamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...tamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 | 1 - ...tamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b | 1 + ...stamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...tamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 | 1 - .../spark/sql/hive/execution/HiveQuerySuite.scala | 14 ++++++-------- 9 files changed, 8 insertions(+), 10 deletions(-) rename sql/hive/src/test/resources/golden/{constant null testing-1-237a6af90a857da1efcbe98f6bbbf9d6 => constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/constant null testing-0-dce645ca43d2bf888c4d75c456e42f3 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/constant null testing-1-237a6af90a857da1efcbe98f6bbbf9d6 b/sql/hive/src/test/resources/golden/constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b similarity index 100% rename from sql/hive/src/test/resources/golden/constant null testing-1-237a6af90a857da1efcbe98f6bbbf9d6 rename to sql/hive/src/test/resources/golden/constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/constant null testing-0-dce645ca43d2bf888c4d75c456e42f3 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f b/sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f new file mode 100644 index 000000000000..d3827e75a5ca --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da887..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b b/sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b new file mode 100644 index 000000000000..18e16e38c5de --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b @@ -0,0 +1 @@ +-1.0 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 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 94660b565893..8437b296feb1 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 @@ -180,8 +180,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd "SELECT CAST(CAST('NaN' AS DOUBLE) AS DECIMAL(1,1)) FROM src LIMIT 1") createQueryTest("constant null testing", - """set spark.sql.legacy.allowCastNumericToTimestamp=true; - | SELECT + """| SELECT |IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL1, |IF(TRUE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2, |IF(FALSE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL3, @@ -202,13 +201,14 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), TIMESTAMP_SECONDS(1)) AS COL20") + val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP)," + + "CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL20") .collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } @@ -564,14 +564,12 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd createQueryTest("timestamp cast #3", """ - |set spark.sql.legacy.allowCastNumericToTimestamp=true; - |SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 + |SELECT CAST(TIMESTAMP_SECONDS(CAST(1.2 AS INT)) AS DOUBLE) FROM src LIMIT 1 """.stripMargin) createQueryTest("timestamp cast #4", """ - |set spark.sql.legacy.allowCastNumericToTimestamp=true; - |SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 + |SELECT CAST(TIMESTAMP_SECONDS(CAST(-1.2 AS INT)) AS DOUBLE) FROM src LIMIT 1 """.stripMargin) test("timestamp cast #5") { From ffa3079cb4e52ce806cce15907f1379e2bf7e646 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 16 Jun 2020 00:37:43 +0800 Subject: [PATCH 72/75] remove unnecessary commit --- .../src/test/resources/spark-version-info.properties | 6 ++++++ .../spark/sql/hive/execution/HiveQuerySuite.scala | 11 ++++++----- 2 files changed, 12 insertions(+), 5 deletions(-) create mode 100644 sql/hive/src/test/resources/spark-version-info.properties diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties new file mode 100644 index 000000000000..69db78bdf332 --- /dev/null +++ b/sql/hive/src/test/resources/spark-version-info.properties @@ -0,0 +1,6 @@ +version=3.1.0 +user=dcadmin +revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f +branch=31710-fix-compatibility +date=2020-06-03T13:50:55Z +url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file 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 8437b296feb1..82355124cae5 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 @@ -180,7 +180,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd "SELECT CAST(CAST('NaN' AS DOUBLE) AS DECIMAL(1,1)) FROM src LIMIT 1") createQueryTest("constant null testing", - """| SELECT + """SELECT |IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL1, |IF(TRUE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2, |IF(FALSE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL3, @@ -205,11 +205,12 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) - test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP)," + - "CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL20") - .collect().head + var r1 = sql( + """ + |SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), + |CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL20 + """.stripMargin).collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } From bc4b62c4522c073dcf5ab3a3b430feff8365c12d Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 16 Jun 2020 00:38:50 +0800 Subject: [PATCH 73/75] remove test file --- sql/hive/src/test/resources/spark-version-info.properties | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 sql/hive/src/test/resources/spark-version-info.properties diff --git a/sql/hive/src/test/resources/spark-version-info.properties b/sql/hive/src/test/resources/spark-version-info.properties deleted file mode 100644 index 69db78bdf332..000000000000 --- a/sql/hive/src/test/resources/spark-version-info.properties +++ /dev/null @@ -1,6 +0,0 @@ -version=3.1.0 -user=dcadmin -revision=2344e1ffb79beaf5e646dfc8b5b916affe7cef7f -branch=31710-fix-compatibility -date=2020-06-03T13:50:55Z -url=https://github.com/GuoPhilipse/spark.git \ No newline at end of file From 08aee309680e95c6a247eab29fd8b51500527c99 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 16 Jun 2020 01:08:55 +0800 Subject: [PATCH 74/75] improve test case --- .../timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f | 1 - .../timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 | 0 .../timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 | 1 + .../timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b | 1 - .../timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 | 0 .../timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 | 1 + .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 6 ++++-- 7 files changed, 6 insertions(+), 4 deletions(-) delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f b/sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f deleted file mode 100644 index d3827e75a5ca..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #3-0-89ef5480d0dd001cec4642de58f8632f +++ /dev/null @@ -1 +0,0 @@ -1.0 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 new file mode 100644 index 000000000000..5625e59da887 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b b/sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b deleted file mode 100644 index 18e16e38c5de..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-0-5a1f6c48122aae7d90b30df172fa9e5b +++ /dev/null @@ -1 +0,0 @@ --1.0 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 new file mode 100644 index 000000000000..1d94c8a014fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 @@ -0,0 +1 @@ +-1.2 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 82355124cae5..893f45e8f0b8 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 @@ -565,12 +565,14 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd createQueryTest("timestamp cast #3", """ - |SELECT CAST(TIMESTAMP_SECONDS(CAST(1.2 AS INT)) AS DOUBLE) FROM src LIMIT 1 + |set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 """.stripMargin) createQueryTest("timestamp cast #4", """ - |SELECT CAST(TIMESTAMP_SECONDS(CAST(-1.2 AS INT)) AS DOUBLE) FROM src LIMIT 1 + |set spark.sql.legacy.allowCastNumericToTimestamp=true; + |SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 """.stripMargin) test("timestamp cast #5") { From 12b42396f058569354040d466962904794fa5c5e Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 16 Jun 2020 01:23:00 +0800 Subject: [PATCH 75/75] improve test case --- ...mp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...p cast #3-1-732ed232ac592c5e7f7c913a88874fd2 | 1 - ...mp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 | 0 ...p cast #4-1-6d2da5cfada03605834e38bc4075bc79 | 1 - .../sql/hive/execution/HiveQuerySuite.scala | 17 +++-------------- 5 files changed, 3 insertions(+), 16 deletions(-) delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-dce645ca43d2bf888c4d75c456e42f3 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da887..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #3-1-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-dce645ca43d2bf888c4d75c456e42f3 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-1-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 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 893f45e8f0b8..2b42444ceeaa 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 @@ -205,6 +205,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) + test("constant null testing timestamp") { var r1 = sql( """ @@ -563,24 +564,12 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assert(-1 == res.get(0)) } - createQueryTest("timestamp cast #3", - """ - |set spark.sql.legacy.allowCastNumericToTimestamp=true; - |SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 - """.stripMargin) - - createQueryTest("timestamp cast #4", - """ - |set spark.sql.legacy.allowCastNumericToTimestamp=true; - |SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1 - """.stripMargin) - - test("timestamp cast #5") { + test("timestamp cast #3") { val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } - test("timestamp cast #6") { + test("timestamp cast #4") { val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1200) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) }