Skip to content

Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp" #27351

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

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all 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
2 changes: 0 additions & 2 deletions R/pkg/R/functions.R
Original file line number Diff line number Diff line change
Expand Up @@ -2496,7 +2496,6 @@ setMethod("schema_of_csv", signature(x = "characterOrColumn"),
#' @note from_utc_timestamp since 1.5.0
setMethod("from_utc_timestamp", signature(y = "Column", x = "character"),
function(y, x) {
.Deprecated(msg = "from_utc_timestamp is deprecated. See SPARK-25496.")
jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x)
column(jc)
})
Expand Down Expand Up @@ -2555,7 +2554,6 @@ setMethod("next_day", signature(y = "Column", x = "character"),
#' @note to_utc_timestamp since 1.5.0
setMethod("to_utc_timestamp", signature(y = "Column", x = "character"),
function(y, x) {
.Deprecated(msg = "to_utc_timestamp is deprecated. See SPARK-25496.")
jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x)
column(jc)
})
Expand Down
18 changes: 4 additions & 14 deletions R/pkg/tests/fulltests/test_sparkSQL.R
Original file line number Diff line number Diff line change
Expand Up @@ -1907,20 +1907,10 @@ test_that("date functions on a DataFrame", {
df2 <- createDataFrame(l2)
expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24))
expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34))
conf <- callJMethod(sparkSession, "conf")
isUtcTimestampFuncEnabled <- callJMethod(conf, "get", "spark.sql.legacy.utcTimestampFunc.enabled")
callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", "true")
tryCatch({
# Both from_utc_timestamp and to_utc_timestamp are deprecated as of SPARK-25496
expect_equal(suppressWarnings(collect(select(df2, from_utc_timestamp(df2$b, "JST"))))[, 1],
c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC")))
expect_equal(suppressWarnings(collect(select(df2, to_utc_timestamp(df2$b, "JST"))))[, 1],
c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC")))
},
finally = {
# Reverting the conf back
callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", isUtcTimestampFuncEnabled)
})
expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1],
c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC")))
expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1],
c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC")))
expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0)
expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0)
expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0)
Expand Down
10 changes: 0 additions & 10 deletions python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -1311,10 +1311,7 @@ def from_utc_timestamp(timestamp, tz):
[Row(local_time=datetime.datetime(1997, 2, 28, 2, 30))]
>>> df.select(from_utc_timestamp(df.ts, df.tz).alias('local_time')).collect()
[Row(local_time=datetime.datetime(1997, 2, 28, 19, 30))]

.. note:: Deprecated in 3.0. See SPARK-25496
"""
warnings.warn("Deprecated in 3.0. See SPARK-25496", DeprecationWarning)
sc = SparkContext._active_spark_context
if isinstance(tz, Column):
tz = _to_java_column(tz)
Expand Down Expand Up @@ -1348,10 +1345,7 @@ def to_utc_timestamp(timestamp, tz):
[Row(utc_time=datetime.datetime(1997, 2, 28, 18, 30))]
>>> df.select(to_utc_timestamp(df.ts, df.tz).alias('utc_time')).collect()
[Row(utc_time=datetime.datetime(1997, 2, 28, 1, 30))]

.. note:: Deprecated in 3.0. See SPARK-25496
"""
warnings.warn("Deprecated in 3.0. See SPARK-25496", DeprecationWarning)
sc = SparkContext._active_spark_context
if isinstance(tz, Column):
tz = _to_java_column(tz)
Expand Down Expand Up @@ -2955,13 +2949,9 @@ def _test():
globs['sc'] = sc
globs['spark'] = spark
globs['df'] = spark.createDataFrame([Row(name='Alice', age=2), Row(name='Bob', age=5)])

spark.conf.set("spark.sql.legacy.utcTimestampFunc.enabled", "true")
(failure_count, test_count) = doctest.testmod(
pyspark.sql.functions, globs=globs,
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
spark.conf.unset("spark.sql.legacy.utcTimestampFunc.enabled")

spark.stop()
if failure_count:
sys.exit(-1)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}

Expand Down Expand Up @@ -1153,19 +1152,11 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
> SELECT _FUNC_('2016-08-31', 'Asia/Seoul');
2016-08-31 09:00:00
""",
since = "1.5.0",
deprecated = """
Deprecated since 3.0.0. See SPARK-25496.
""")
since = "1.5.0")
// scalastyle:on line.size.limit
case class FromUTCTimestamp(left: Expression, right: Expression)
extends BinaryExpression with ImplicitCastInputTypes {

if (!SQLConf.get.utcTimestampFuncEnabled) {
throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0." +
s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.")
}

override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType)
override def dataType: DataType = TimestampType
override def prettyName: String = "from_utc_timestamp"
Expand Down Expand Up @@ -1367,19 +1358,11 @@ case class MonthsBetween(
> SELECT _FUNC_('2016-08-31', 'Asia/Seoul');
2016-08-30 15:00:00
""",
since = "1.5.0",
deprecated = """
Deprecated since 3.0.0. See SPARK-25496.
""")
since = "1.5.0")
// scalastyle:on line.size.limit
case class ToUTCTimestamp(left: Expression, right: Expression)
extends BinaryExpression with ImplicitCastInputTypes {

if (!SQLConf.get.utcTimestampFuncEnabled) {
throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0. " +
s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.")
}

override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType)
override def dataType: DataType = TimestampType
override def prettyName: String = "to_utc_timestamp"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2038,12 +2038,6 @@ object SQLConf {
.booleanConf
.createWithDefault(false)

val UTC_TIMESTAMP_FUNC_ENABLED = buildConf("spark.sql.legacy.utcTimestampFunc.enabled")
.doc("The configuration property enables the to_utc_timestamp() " +
"and from_utc_timestamp() functions.")
.booleanConf
.createWithDefault(false)

val SOURCES_BINARY_FILE_MAX_LENGTH = buildConf("spark.sql.sources.binaryFile.maxLength")
.doc("The max length of a file that can be read by the binary file data source. " +
"Spark will fail fast and not attempt to read the file if its length exceeds this value. " +
Expand Down Expand Up @@ -2423,8 +2417,6 @@ class SQLConf extends Serializable with Logging {

def datetimeJava8ApiEnabled: Boolean = getConf(DATETIME_JAVA8API_ENABLED)

def utcTimestampFuncEnabled: Boolean = getConf(UTC_TIMESTAMP_FUNC_ENABLED)

def addDirectoryRecursiveEnabled: Boolean = getConf(LEGACY_ADD_DIRECTORY_USING_RECURSIVE)

def legacyMsSqlServerNumericMappingEnabled: Boolean =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.expressions.objects._
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.ThreadUtils
Expand Down Expand Up @@ -187,42 +186,36 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper {
}

test("SPARK-17702: split wide constructor into blocks due to JVM code size limit") {
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
val length = 5000
val expressions = Seq.fill(length) {
ToUTCTimestamp(
Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType),
Literal.create("PST", StringType))
}
val plan = GenerateMutableProjection.generate(expressions)
val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType))
val expected = Seq.fill(length)(
DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00")))

if (actual != expected) {
fail(
s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected")
}
val length = 5000
val expressions = Seq.fill(length) {
ToUTCTimestamp(
Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType),
Literal.create("PST", StringType))
}
val plan = GenerateMutableProjection.generate(expressions)
val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType))
val expected = Seq.fill(length)(
DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00")))

if (actual != expected) {
fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected")
}
}

test("SPARK-22226: group splitted expressions into one method per nested class") {
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
val length = 10000
val expressions = Seq.fill(length) {
ToUTCTimestamp(
Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType),
Literal.create("PST", StringType))
}
val plan = GenerateMutableProjection.generate(expressions)
val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType))
val expected = Seq.fill(length)(
DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2017-10-10 07:00:00")))

if (actual != expected) {
fail(
s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected")
}
val length = 10000
val expressions = Seq.fill(length) {
ToUTCTimestamp(
Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType),
Literal.create("PST", StringType))
}
val plan = GenerateMutableProjection.generate(expressions)
val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType))
val expected = Seq.fill(length)(
DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2017-10-10 07:00:00")))

if (actual != expected) {
fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,12 @@ import java.util.concurrent.TimeUnit
import java.util.concurrent.TimeUnit._

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}

Expand Down Expand Up @@ -894,29 +892,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
NonFoldableLiteral.create(tz, StringType)),
if (expected != null) Timestamp.valueOf(expected) else null)
}
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00")
test("2015-01-24 00:00:00", "PST", "2015-01-24 08:00:00")
test(null, "UTC", null)
test("2015-07-24 00:00:00", null, null)
test(null, null, null)
}
val msg = intercept[AnalysisException] {
test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00")
}.getMessage
assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key))
test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00")
test("2015-01-24 00:00:00", "PST", "2015-01-24 08:00:00")
test(null, "UTC", null)
test("2015-07-24 00:00:00", null, null)
test(null, null, null)
}

test("to_utc_timestamp - invalid time zone id") {
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz =>
val msg = intercept[java.time.DateTimeException] {
GenerateUnsafeProjection.generate(
ToUTCTimestamp(
Literal(Timestamp.valueOf("2015-07-24 00:00:00")), Literal(invalidTz)) :: Nil)
}.getMessage
assert(msg.contains(invalidTz))
}
Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz =>
val msg = intercept[java.time.DateTimeException] {
GenerateUnsafeProjection.generate(
ToUTCTimestamp(
Literal(Timestamp.valueOf("2015-07-24 00:00:00")), Literal(invalidTz)) :: Nil)
}.getMessage
assert(msg.contains(invalidTz))
}
}

Expand All @@ -933,28 +923,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
NonFoldableLiteral.create(tz, StringType)),
if (expected != null) Timestamp.valueOf(expected) else null)
}
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00")
test("2015-01-24 00:00:00", "PST", "2015-01-23 16:00:00")
test(null, "UTC", null)
test("2015-07-24 00:00:00", null, null)
test(null, null, null)
}
val msg = intercept[AnalysisException] {
test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00")
}.getMessage
assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key))
test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00")
test("2015-01-24 00:00:00", "PST", "2015-01-23 16:00:00")
test(null, "UTC", null)
test("2015-07-24 00:00:00", null, null)
test(null, null, null)
}

test("from_utc_timestamp - invalid time zone id") {
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz =>
val msg = intercept[java.time.DateTimeException] {
GenerateUnsafeProjection.generate(
FromUTCTimestamp(Literal(0), Literal(invalidTz)) :: Nil)
}.getMessage
assert(msg.contains(invalidTz))
}
Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz =>
val msg = intercept[java.time.DateTimeException] {
GenerateUnsafeProjection.generate(FromUTCTimestamp(Literal(0), Literal(invalidTz)) :: Nil)
}.getMessage
assert(msg.contains(invalidTz))
}
}

Expand Down
4 changes: 0 additions & 4 deletions sql/core/src/main/scala/org/apache/spark/sql/functions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3053,7 +3053,6 @@ object functions {
* @group datetime_funcs
* @since 1.5.0
*/
@deprecated("This function is deprecated and will be removed in future versions.", "3.0.0")
def from_utc_timestamp(ts: Column, tz: String): Column = withExpr {
FromUTCTimestamp(ts.expr, Literal(tz))
}
Expand All @@ -3065,7 +3064,6 @@ object functions {
* @group datetime_funcs
* @since 2.4.0
*/
@deprecated("This function is deprecated and will be removed in future versions.", "3.0.0")
def from_utc_timestamp(ts: Column, tz: Column): Column = withExpr {
FromUTCTimestamp(ts.expr, tz.expr)
}
Expand All @@ -3084,7 +3082,6 @@ object functions {
* @group datetime_funcs
* @since 1.5.0
*/
@deprecated("This function is deprecated and will be removed in future versions.", "3.0.0")
def to_utc_timestamp(ts: Column, tz: String): Column = withExpr {
ToUTCTimestamp(ts.expr, Literal(tz))
}
Expand All @@ -3096,7 +3093,6 @@ object functions {
* @group datetime_funcs
* @since 2.4.0
*/
@deprecated("This function is deprecated and will be removed in future versions.", "3.0.0")
def to_utc_timestamp(ts: Column, tz: Column): Column = withExpr {
ToUTCTimestamp(ts.expr, tz.expr)
}
Expand Down
Loading