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-31797][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions #28534

Closed
wants to merge 18 commits into from

Conversation

TJX2014
Copy link
Contributor

@TJX2014 TJX2014 commented May 15, 2020

What changes were proposed in this pull request?

Add and register three new functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS
A test is added.

Reference: BigQuery

Why are the changes needed?

People will have convenient way to get timestamps from seconds,milliseconds and microseconds.

Does this PR introduce any user-facing change?

Yes, people will have the following ways to get timestamp:

sql("select TIMESTAMP_SECONDS(t.a) as timestamp from values(1230219000),(-1230219000) as t(a)").show(false)
+-------------------------+
|timestamp                  |
+-------------------------+
|2008-12-25 23:30:00|
|1931-01-07 16:30:00|
+-------------------------+
sql("select TIMESTAMP_MILLIS(t.a) as timestamp from values(1230219000123),(-1230219000123) as t(a)").show(false)
+-------------------------------+
|timestamp                           |
+-------------------------------+
|2008-12-25 23:30:00.123|
|1931-01-07 16:29:59.877|
+-------------------------------+
sql("select TIMESTAMP_MICROS(t.a) as timestamp from values(1230219000123123),(-1230219000123123) as t(a)").show(false)
+------------------------------------+
|timestamp                                   |
+------------------------------------+
|2008-12-25 23:30:00.123123|
|1931-01-07 16:29:59.876877|
+------------------------------------+

How was this patch tested?

Unit test.

@bart-samwel
Copy link

This is a behavior change for existing Spark workloads. Even if the code is not compatible with Hive here, we shouldn't break existing workloads. See also the versioning policy.. If we really want to fix this, we would have to introduce a compatibility flag, and fail this construct by default so that people can choose. A better solution would be to introduce functions that are explicit about the meaning of their source operand. E.g. this is how it is done in BigQuery, with functions like TIMESTAMP_MILLIS to get a timestamp from a Unix miliseconds timestamp.

@cloud-fan @MaxGekk FYI

@TJX2014
Copy link
Contributor Author

TJX2014 commented May 15, 2020

@bart-samwel Nice suggestion, I am both ok in conform to BigQuery or Hive.

@cloud-fan
Copy link
Contributor

This is not standard behavior. It's OK to follow Hive, but this behavior has been released a long time ago, and is the behavior of Spark for many years. It's too late to change now.

Agree with @bart-samwel to add a dedicated function to convert milli/micro seconds to timestamp.

@TJX2014
Copy link
Contributor Author

TJX2014 commented May 15, 2020

@cloud-fan @bart-samwel Thanks, I will make a revert and to add a dedicated function to support it.

@TJX2014 TJX2014 force-pushed the master-SPARK-31710 branch 2 times, most recently from 487197b to 39eac6c Compare May 16, 2020 10:20
@TJX2014 TJX2014 changed the title [SPARK-31710][SQL]Fix timestamp and long convert not compatible with hive issue [SPARK-31710][SQL]Fix millisecond and microsecond convert to timestamp in to_timestamp May 16, 2020
@TJX2014
Copy link
Contributor Author

TJX2014 commented May 16, 2020

@cloud-fan @MaxGekk Could you please help me review it ?

@cloud-fan
Copy link
Contributor

ok to test

@SparkQA
Copy link

SparkQA commented May 17, 2020

Test build #122749 has finished for PR 28534 at commit 48e81cc.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented May 18, 2020

Test build #122765 has finished for PR 28534 at commit 7d562f4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@TJX2014
Copy link
Contributor Author

TJX2014 commented May 18, 2020

Test build #122765 has finished for PR 28534 at commit 7d562f4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan Could you please verify this patch

withSQLConf() {
checkEvaluation(
GetTimestamp(
Literal(1580184371847000L),

Choose a reason for hiding this comment

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

Test with literals that don't end in three 0s?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok, I will replace three 0s with 123 in order to avoid mistake

Literal("milli")), 1580184371847000000L)
checkEvaluation(
GetTimestamp(
Literal(1580184371847000L),

Choose a reason for hiding this comment

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

Also test with negative (pre-Unix epoch) timestamps.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You are right, I will add negative test.

checkExceptionInExpression[IllegalArgumentException](
GetTimestamp(
Literal(1580184371847000L),
Literal("other")),

Choose a reason for hiding this comment

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

Also test with a non-literal input. The error should still make sense then.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@bart-samwel Could you please give me an example about what a non-literal means.

@@ -846,6 +853,8 @@ abstract class ToTimestamp
case NonFatal(_) => null
}
}
case LongType =>
Math.multiplyExact(t.asInstanceOf[Long], scaleFactor.asInstanceOf[Long])

Choose a reason for hiding this comment

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

@cloud-fan Do we do range checks on timestamps? (Do we even have published valid ranges?) Because if we do, then this could produce valid Long values that are out of range.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I will add a check when decide the value of scaleFactor to avoid the result of multiply out of range of Long value.

@SparkQA
Copy link

SparkQA commented May 18, 2020

Test build #122799 has finished for PR 28534 at commit 019d6b2.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor Author

@TJX2014 TJX2014 left a comment

Choose a reason for hiding this comment

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

correct some issue base on suggestions

withSQLConf() {
checkEvaluation(
GetTimestamp(
Literal(1580184371847000L),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok, I will replace three 0s with 123 in order to avoid mistake

@@ -846,6 +853,8 @@ abstract class ToTimestamp
case NonFatal(_) => null
}
}
case LongType =>
Math.multiplyExact(t.asInstanceOf[Long], scaleFactor.asInstanceOf[Long])
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I will add a check when decide the value of scaleFactor to avoid the result of multiply out of range of Long value.

checkExceptionInExpression[IllegalArgumentException](
GetTimestamp(
Literal(1580184371847000L),
Literal("other")),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@bart-samwel Could you please give me an example about what a non-literal means.

Literal("milli")), 1580184371847000000L)
checkEvaluation(
GetTimestamp(
Literal(1580184371847000L),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

You are right, I will add negative test.

@TJX2014
Copy link
Contributor Author

TJX2014 commented May 18, 2020

@bart-samwel a patch is added, Could you please help me review it.

case LongType =>
val input = t.asInstanceOf[Long]
if ( minRange < input && input < maxRange ) {
Math.multiplyExact(input, scaleFactor.asInstanceOf[Long])
Copy link
Member

Choose a reason for hiding this comment

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

Math.multiplyExact throws ArithmeticException. You don't need to explicitly check the ranges.

Choose a reason for hiding this comment

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

My earlier concern was about whether we support the full range of Long as valid timestamps, or if the actual supported range is smaller. I googled a bit, and it seems that the full range is supported. So that's fine then without checks.

@@ -806,6 +806,17 @@ abstract class ToTimestamp
case NonFatal(_) => null
}

private lazy val scaleFactor = right.toString match {
Copy link
Member

Choose a reason for hiding this comment

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

Please, look at other places in the file how to handle foldable expressions.

@@ -789,7 +789,7 @@ abstract class ToTimestamp
protected def downScaleFactor: Long

override def inputTypes: Seq[AbstractDataType] =
Seq(TypeCollection(StringType, DateType, TimestampType), StringType)
Seq(TypeCollection(StringType, DateType, TimestampType, LongType ), StringType)
Copy link
Member

Choose a reason for hiding this comment

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

Why only LongType. Can be any IntegralType

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, IntegerType should also be considered.

$javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
if (!${ev.isNull}) {
if ( (${minRange}L < ${eval1.value}) && (${eval1.value} < ${maxRange}L) ) {
${ev.value} = ${eval1.value} * $scaleFactor;
Copy link
Member

Choose a reason for hiding this comment

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

Inconsistent with non-codegen where you use Math.multiplyExact

@@ -846,6 +857,15 @@ abstract class ToTimestamp
case NonFatal(_) => null
}
}
case LongType =>
val input = t.asInstanceOf[Long]
if ( minRange < input && input < maxRange ) {
Copy link
Member

Choose a reason for hiding this comment

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

Just in case, why do you exclude min and max values?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, min and max should also be considered as correct input.Thanks for your advise.

Comment on lines 956 to 957
"input [" + ${eval1.value} + "] not from " + ${minRange}L + " to "
+ ${maxRange}L + " for format " + "${right.toString}");
Copy link
Member

Choose a reason for hiding this comment

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

hmm, does this string interpolation work in Java?

test("SPARK-31710:Fix millisecond and microsecond convert to timestamp in to_timestamp") {
withSQLConf() {
checkEvaluation(
GetTimestamp(
Copy link
Contributor

@cloud-fan cloud-fan May 18, 2020

Choose a reason for hiding this comment

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

I thought we are going to add 3 new functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS, to follow big query. Why do we overload GetTimestamp? Passing unit string as parameter is fragile, as we need to define/document the supported units and the behavior of invalid units.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think your idea is right, I will add the three functions later and make a revert.

@bart-samwel
Copy link

I'd like to cross reference this with PR #28568 which is trying to solve this same issue with a compatibility flag. I think those two solutions are orthogonal -- there can be a flag to change the behavior (to easy migration from Hive), and there can be a better way to specify the conversion type explicitly. So I think the direction of this PR should go in no matter what happens with the other PR.

@SparkQA
Copy link

SparkQA commented May 18, 2020

Test build #122804 has finished for PR 28534 at commit c2be46b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@TJX2014 TJX2014 changed the title [SPARK-31710][SQL]Fix millisecond and microsecond convert to timestamp in to_timestamp [SPARK-31710][SQL]TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS to timestamp transfer May 19, 2020
@TJX2014 TJX2014 requested a review from cloud-fan May 19, 2020 12:47
@@ -401,6 +401,92 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas
}
}

@ExpressionDescription(
usage = "_FUNC_(date) - Returns timestamp from seconds.",
Copy link
Contributor

Choose a reason for hiding this comment

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

_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.

}

@ExpressionDescription(
usage = "_FUNC_(date) - Returns timestamp from milliseconds.",
Copy link
Contributor

Choose a reason for hiding this comment

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

_FUNC_(seconds) - Creates timestamp from the number of milliseconds since UTC epoch.

}

@ExpressionDescription(
usage = "_FUNC_(date) - Returns timestamp from microseconds.",
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto


protected def upScaleFactor: Long

override def inputTypes: Seq[AbstractDataType] = Seq(LongType, IntegerType)
Copy link
Contributor

Choose a reason for hiding this comment

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

we should accept LongType only. The analyzer will cast the input to long for you, if possible.

@SparkQA
Copy link

SparkQA commented May 22, 2020

Test build #122972 has finished for PR 28534 at commit 7a54ffb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented May 22, 2020

Test build #122974 has finished for PR 28534 at commit fba8827.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented May 22, 2020

Test build #122987 has finished for PR 28534 at commit f0cc631.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in 2115c55 May 22, 2020
@cloud-fan
Copy link
Contributor

Hi @TJX2014 , the original JIRA doesn't fit this PR very well. Can you create a new JIRA ticket and update the PR title? Thanks!

@TJX2014
Copy link
Contributor Author

TJX2014 commented May 22, 2020

Thanks very much. Got a lot.

@TJX2014
Copy link
Contributor Author

TJX2014 commented May 22, 2020

Hi @TJX2014 , the original JIRA doesn't fit this PR very well. Can you create a new JIRA ticket and update the PR title? Thanks!

@cloud-fan ok, I will create the jira.

@TJX2014 TJX2014 changed the title [SPARK-31710][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions [SPARK-31797][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions May 22, 2020
dongjoon-hyun pushed a commit that referenced this pull request Jul 1, 2020
### What changes were proposed in this pull request?

This is a followup of #28534 , to make `TIMESTAMP_SECONDS` function support fractional input as well.

### Why are the changes needed?

Previously the cast function can cast fractional values to timestamp. Now we suggest users to ues these new functions, and we need to cover all the cast use cases.

### Does this PR introduce _any_ user-facing change?

Yes, now `TIMESTAMP_SECONDS` function accepts fractional input.

### How was this patch tested?

new tests

Closes #28956 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
dongjoon-hyun pushed a commit that referenced this pull request Dec 2, 2020
…X_MICROS

### What changes were proposed in this pull request?

As #28534 adds functions from [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/timestamp_functions) for converting numbers to timestamp, this PR is to add functions UNIX_SECONDS, UNIX_MILLIS and UNIX_MICROS for converting timestamp to numbers.

### Why are the changes needed?

1. Symmetry of the conversion functions
2. Casting timestamp type to numeric types is disallowed in ANSI mode, we should provide functions for users to complete the conversion.

### Does this PR introduce _any_ user-facing change?

3 new functions UNIX_SECONDS, UNIX_MILLIS and UNIX_MICROS for converting timestamp to long type.

### How was this patch tested?

Unit tests.

Closes #30566 from gengliangwang/timestampLong.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants