Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-33549][SQL] Remove configuration spark.sql.legacy.allowCastNumericToTimestamp #30493

Closed
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -59,8 +59,7 @@ object Cast {
case (StringType, TimestampType) => true
case (BooleanType, TimestampType) => true
case (DateType, TimestampType) => true
case (_: NumericType, TimestampType) =>
SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP)
case (_: NumericType, TimestampType) => true

case (StringType, DateType) => true
case (TimestampType, DateType) => true
Expand Down Expand Up @@ -273,15 +272,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
TypeCheckResult.TypeCheckSuccess
} else {
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 " +
s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," +
"but we strongly recommend using function " +
"TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead."
} else {
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}"
})
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2824,15 +2824,6 @@ 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 casting numeric to timestamp," +
"when false, forbid the cast, more details in SPARK-31710")
.version("3.1.0")
.booleanConf
.createWithDefault(true)

val COALESCE_BUCKETS_IN_JOIN_ENABLED =
buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled")
.doc("When true, if two bucketed tables with the different number of buckets are joined, " +
Expand Down Expand Up @@ -3550,9 +3541,6 @@ class SQLConf extends Serializable with Logging {

def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID)

def legacyAllowCastNumericToTimestamp: Boolean =
getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP)

def metadataCacheTTL: Long = getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS)

def coalesceBucketsInJoinEnabled: Boolean = getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1311,20 +1311,6 @@ class CastSuite extends CastSuiteBase {
}
}

test("SPARK-31710: fail casting from numeric to timestamp if it is forbidden") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Users just need to set spark.sql.ansi.enabled for the behavior.

Just in case, should we leave the test, and check the cases under ^^ the config?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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)
}
}
}

test("SPARK-32828: cast from a derived user-defined type to a base type") {
val v = Literal.create(Row(1), new ExampleSubTypeUDT())
checkEvaluation(cast(v, new ExampleBaseTypeUDT), Row(1))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,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")
// 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()
}

Expand All @@ -73,8 +71,6 @@ 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())
Expand Down