Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
18 commits
Select commit Hold shift + click to select a range
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
Prev Previous commit
Next Next commit
improve code quality according to advices
  • Loading branch information
TJX2014 committed May 22, 2020
commit 27a162b32ecdd824f720c757ee6e5090ebf43669
Original file line number Diff line number Diff line change
Expand Up @@ -402,11 +402,11 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas
}

@ExpressionDescription(
usage = "_FUNC_(date) - Returns timestamp from seconds.",
usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.",
examples = """
Examples:
> SELECT _FUNC_(1230219000);
"2008-12-25 07:30:00.0"
"2008-12-25 07:30:00"
""",
group = "datetime_funcs",
since = "3.1.0")
Expand All @@ -419,11 +419,12 @@ case class SecondsToTimestamp(child: Expression)
}

@ExpressionDescription(
usage = "_FUNC_(date) - Returns timestamp from milliseconds.",
usage = "_FUNC_(milliseconds) - " +
"Creates timestamp from the number of milliseconds since UTC epoch.",
Copy link
Member

Choose a reason for hiding this comment

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

nit: let's do either:

usage = """
  _FUNC_(milliseconds) - Creates timestamp from the number of milliseconds
    since UTC epoch.
    ...
"""

or

// scalastyle:off line.size.limit
@ExpressionDescription(
  usage = "_FUNC_(milliseconds) - Creates timestamp from the number of milliseconds since UTC epoch.",
  ...
// scalastyle:on line.size.limit

just for the sake of consistency.

examples = """
Examples:
> SELECT _FUNC_(1230219000000);

Choose a reason for hiding this comment

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

Include precision in the example, e.g.

Suggested change
> SELECT _FUNC_(1230219000000);
> SELECT _FUNC_(12302190000123);
"2008-12-25 07:30:00.123"

Same for micros below.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

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, datetime.sql is also syncronized.

"2008-12-25 07:30:00.0"
"2008-12-25 07:30:00"
""",
group = "datetime_funcs",
since = "3.1.0")
Expand All @@ -436,11 +437,12 @@ case class MilliSecondsToTimestamp(child: Expression)
}

@ExpressionDescription(
usage = "_FUNC_(date) - Returns timestamp from microseconds.",
usage = "_FUNC_(microseconds) - " +
"Creates timestamp from the number of microseconds since UTC epoch.",
examples = """
Examples:
> SELECT _FUNC_(1230219000000000);
"2008-12-25 07:30:00.0"
"2008-12-25 07:30:00"
""",
group = "datetime_funcs",
since = "3.1.0")
Expand All @@ -457,29 +459,19 @@ abstract class NumberToTimestampBase extends UnaryExpression

protected def upScaleFactor: Long

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

override def dataType: DataType = TimestampType

override def eval(input: InternalRow): Any = {
val t = child.eval(input)
if (t == null) {
null
} else {
child.dataType match {
case IntegerType =>
Math.multiplyExact(t.asInstanceOf[Int].toLong, upScaleFactor)
case LongType =>
Math.multiplyExact(t.asInstanceOf[Long], upScaleFactor)
}
override def nullSafeEval(input: Any): Any = {
child.dataType match {
Copy link
Contributor

Choose a reason for hiding this comment

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

we don't need to check the type. The analyzer should guarantee it. We can just write

Math.multiplyExact(input.asInstanceOf[Long], upScaleFactor)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Cool

case LongType =>
Math.multiplyExact(input.asInstanceOf[Long], upScaleFactor)
}
}

override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
child.dataType match {
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

case IntegerType =>
defineCodeGen(ctx, ev, _ => s"java.lang.Math.multiplyExact(" +
s"${ev.value.asInstanceOf[Integer].toLong}, ${upScaleFactor})")
case LongType =>
defineCodeGen(ctx, ev, _ => s"java.lang.Math.multiplyExact(" +
s"${ev.value.asInstanceOf[Long]}, ${upScaleFactor})")
Copy link
Contributor

Choose a reason for hiding this comment

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

this should be ev.value, otherwise you get class cast exception, as ev.value is a string.

Expand Down
5 changes: 5 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/datetime.sql
Original file line number Diff line number Diff line change
@@ -1,5 +1,10 @@
-- date time functions

-- [SPARK-31710] TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS to timestamp transfer
select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null);
select TIMESTAMP_MILLISECONDS(1230219000000),TIMESTAMP_MILLISECONDS(-1230219000000),TIMESTAMP_MILLISECONDS(null);
select TIMESTAMP_MICROSECONDS(1230219000000000),TIMESTAMP_MICROSECONDS(-1230219000000000),TIMESTAMP_MICROSECONDS(null);

-- [SPARK-16836] current_date and current_timestamp literals
select current_date = current_date(), current_timestamp = current_timestamp();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,30 @@
-- Number of queries: 91


-- !query
select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
-- !query schema
struct<timestamp_seconds(CAST(1230219000 AS BIGINT)):timestamp,timestamp_seconds(CAST(-1230219000 AS BIGINT)):timestamp,timestamp_seconds(CAST(NULL AS BIGINT)):timestamp>
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@cloud-fan Shall we regenerate it for cast(... as Int) ?

Copy link
Contributor

Choose a reason for hiding this comment

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

we have to, otherwise the test fails

-- !query output
2008-12-25 07:30:00 1931-01-07 00:30:00 NULL


-- !query
select TIMESTAMP_MILLISECONDS(1230219000000),TIMESTAMP_MILLISECONDS(-1230219000000),TIMESTAMP_MILLISECONDS(null)
-- !query schema
struct<timestamp_milliseconds(1230219000000):timestamp,timestamp_milliseconds(-1230219000000):timestamp,timestamp_milliseconds(CAST(NULL AS BIGINT)):timestamp>
-- !query output
2008-12-25 07:30:00 1931-01-07 00:30:00 NULL


-- !query
select TIMESTAMP_MICROSECONDS(1230219000000000),TIMESTAMP_MICROSECONDS(-1230219000000000),TIMESTAMP_MICROSECONDS(null)
-- !query schema
struct<timestamp_microseconds(1230219000000000):timestamp,timestamp_microseconds(-1230219000000000):timestamp,timestamp_microseconds(CAST(NULL AS BIGINT)):timestamp>
-- !query output
2008-12-25 07:30:00 1931-01-07 00:30:00 NULL


-- !query
select current_date = current_date(), current_timestamp = current_timestamp()
-- !query schema
Expand Down
24 changes: 24 additions & 0 deletions sql/core/src/test/resources/sql-tests/results/datetime.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,30 @@
-- Number of queries: 91


-- !query
select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
-- !query schema
struct<timestamp_seconds(CAST(1230219000 AS BIGINT)):timestamp,timestamp_seconds(CAST(-1230219000 AS BIGINT)):timestamp,timestamp_seconds(CAST(NULL AS BIGINT)):timestamp>
-- !query output
2008-12-25 07:30:00 1931-01-07 00:30:00 NULL


-- !query
select TIMESTAMP_MILLISECONDS(1230219000000),TIMESTAMP_MILLISECONDS(-1230219000000),TIMESTAMP_MILLISECONDS(null)
-- !query schema
struct<timestamp_milliseconds(1230219000000):timestamp,timestamp_milliseconds(-1230219000000):timestamp,timestamp_milliseconds(CAST(NULL AS BIGINT)):timestamp>
-- !query output
2008-12-25 07:30:00 1931-01-07 00:30:00 NULL


-- !query
select TIMESTAMP_MICROSECONDS(1230219000000000),TIMESTAMP_MICROSECONDS(-1230219000000000),TIMESTAMP_MICROSECONDS(null)
-- !query schema
struct<timestamp_microseconds(1230219000000000):timestamp,timestamp_microseconds(-1230219000000000):timestamp,timestamp_microseconds(CAST(NULL AS BIGINT)):timestamp>
-- !query output
2008-12-25 07:30:00 1931-01-07 00:30:00 NULL


-- !query
select current_date = current_date(), current_timestamp = current_timestamp()
-- !query schema
Expand Down
32 changes: 14 additions & 18 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3498,24 +3498,20 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark

test("SPARK-31710: " +
Copy link
Contributor

Choose a reason for hiding this comment

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

let's move the tests to datetime.sql

Copy link
Contributor

Choose a reason for hiding this comment

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

shall we remove it? I don't think it provides more test coverage than datetime.sql

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, we shall remove it because it has the same coverage as datetime.sql.

"TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS to timestamp transfer") {
val df1 = sql("select TIMESTAMP_SECONDS(1230219000) as timestamp")
checkAnswer(df1, Row(Timestamp.valueOf("2008-12-25 07:30:00.0")))

val df2 = sql("select TIMESTAMP_MILLISECONDS(1230219000000) as timestamp")
checkAnswer(df2, Row(Timestamp.valueOf("2008-12-25 07:30:00.0")))

val df3 = sql("select TIMESTAMP_MICROSECONDS(1230219000000000) as timestamp")
checkAnswer(df3, Row(Timestamp.valueOf("2008-12-25 07:30:00.0")))

val df4 = sql("select TIMESTAMP_SECONDS(-1230219000) as timestamp")
checkAnswer(df4, Row(Timestamp.valueOf("1931-01-07 00:30:00.0")))

val df5 = sql("select TIMESTAMP_MILLISECONDS(-1230219000000) as timestamp")
checkAnswer(df5, Row(Timestamp.valueOf("1931-01-07 00:30:00.0")))

val df6 = sql("select TIMESTAMP_MICROSECONDS(-1230219000000000) as timestamp")
checkAnswer(df6, Row(Timestamp.valueOf("1931-01-07 00:30:00.0")))

val df1 = sql("select TIMESTAMP_SECONDS(1230219000)," +
"TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)")
checkAnswer(df1, Row(Timestamp.valueOf("2008-12-25 07:30:00"),
Timestamp.valueOf("1931-01-07 00:30:00"), null))

val df2 = sql("select TIMESTAMP_MILLISECONDS(1230219000000)," +
"TIMESTAMP_MILLISECONDS(-1230219000000),TIMESTAMP_MILLISECONDS(null)")
checkAnswer(df2, Row(Timestamp.valueOf("2008-12-25 07:30:00"),
Timestamp.valueOf("1931-01-07 00:30:00"), null))

val df3 = sql("select TIMESTAMP_MICROSECONDS(1230219000000000)," +
"TIMESTAMP_MICROSECONDS(-1230219000000000),TIMESTAMP_MICROSECONDS(null)")
checkAnswer(df3, Row(Timestamp.valueOf("2008-12-25 07:30:00"),
Timestamp.valueOf("1931-01-07 00:30:00"), null))
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 have added these sqls to datetime.sql, but not sure datetime.sql will check the correctness of these functions,so these checks is not removed currently, could you please give me a help and decide if these should retain? @cloud-fan

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@cloud-fan datetime.sql can produce the same results, I mean could this test in SQLQuerySuite be retained or just removed.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It seems we can remove the test in SQLQuerySuite, may be I have a misunderstanding to datetime.sql and the usage.

}
}

Expand Down