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-21739][SQL]Cast expression should initialize timezoneId when it is called statically to convert something into TimestampType #18960

Closed
wants to merge 7 commits into from

Conversation

DonnyZone
Copy link
Contributor

What changes were proposed in this pull request?

https://issues.apache.org/jira/projects/SPARK/issues/SPARK-21739

This issue is caused by introducing TimeZoneAwareExpression.
When the Cast expression converts something into TimestampType, it should be resolved with setting timezoneId. In general, it is resolved in LogicalPlan phase.

However, there are still some places that use Cast expression statically to convert datatypes without setting timezoneId. In such cases, NoSuchElementException: None.get will be thrown for TimestampType.

This PR is proposed to fix the issue. We have checked the whole project and found two such usages(i.e., inTableReader and HiveTableScanExec).

How was this patch tested?

unit test

@DonnyZone
Copy link
Contributor Author

@cloud-fan @gatorsmile

@gatorsmile
Copy link
Member

ok to test

@gatorsmile
Copy link
Member

Also cc @ueshin

@@ -104,7 +105,7 @@ case class HiveTableScanExec(
hadoopConf)

private def castFromString(value: String, dataType: DataType) = {
Cast(Literal(value), dataType).eval(null)
Cast(Literal(value), dataType, Option(SQLConf.get.sessionLocalTimeZone)).eval(null)
Copy link
Member

Choose a reason for hiding this comment

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

Please use CastSupport.cast

Copy link
Contributor Author

Choose a reason for hiding this comment

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

BTW, is it elegant to initialize a CastSupport (DataSourceAnalysis rule or DataSourceStrategy) here, in which we still need to pass SQLConf?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Here, we can obtain SQLConf directly with sparkSession.sessionState.conf

@@ -227,7 +228,8 @@ class HadoopTableReader(
def fillPartitionKeys(rawPartValues: Array[String], row: InternalRow): Unit = {
partitionKeyAttrs.foreach { case (attr, ordinal) =>
val partOrdinal = partitionKeys.indexOf(attr)
row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null)
row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType,
Option(SQLConf.get.sessionLocalTimeZone)).eval(null)
Copy link
Member

Choose a reason for hiding this comment

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

The same here.

Copy link
Member

Choose a reason for hiding this comment

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

We also need a test case for verifying this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Do you mean a test case for HadoopTableReader? a little confusing

@SparkQA
Copy link

SparkQA commented Aug 16, 2017

Test build #80748 has finished for PR 18960 at commit 492b756.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Aug 17, 2017

Test build #80754 has finished for PR 18960 at commit 492b756.

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

@SparkQA
Copy link

SparkQA commented Aug 17, 2017

Test build #80760 has finished for PR 18960 at commit a264e3a.

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

@@ -68,4 +68,25 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl
sql("DROP TABLE IF EXISTS createAndInsertTest")
}
}

test("SPARK-21739: Cast expression should initialize timezoneId " +
Copy link
Member

Choose a reason for hiding this comment

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

This test can pass without the change in TableReader.scala. We need another test case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh, it should select the TimestampType column. Thanks for reminder, I will fix it.

checkAnswer(sql("select value from table_with_timestamp_partition " +
"where ts = '2010-01-02 00:00:00.000'"), Row(2))

sql("DROP TABLE IF EXISTS table_with_timestamp_partition")
Copy link
Member

Choose a reason for hiding this comment

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

use WithTable. You can check how we do it in the other test cases

@DonnyZone
Copy link
Contributor Author

Test case updated.

@DonnyZone
Copy link
Contributor Author

Moreover, how about using CastSupport.cast, shall I initialize a DataSourceAnalysis or DataSourceStrategy ?

@SparkQA
Copy link

SparkQA commented Aug 17, 2017

Test build #80779 has finished for PR 18960 at commit 5a04ddc.

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

@SparkQA
Copy link

SparkQA commented Aug 17, 2017

Test build #80786 has finished for PR 18960 at commit 93581bb.

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

@gatorsmile
Copy link
Member

gatorsmile commented Aug 17, 2017

Could you check the changes I made below? 081d57a

Thanks!

@SparkQA
Copy link

SparkQA commented Aug 18, 2017

Test build #80818 has finished for PR 18960 at commit 5b051dd.

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

@gatorsmile
Copy link
Member

LGTM

@gatorsmile
Copy link
Member

Thanks! Merging to master/2.2

asfgit pushed a commit that referenced this pull request Aug 18, 2017
…it is called statically to convert something into TimestampType

## What changes were proposed in this pull request?

https://issues.apache.org/jira/projects/SPARK/issues/SPARK-21739

This issue is caused by introducing TimeZoneAwareExpression.
When the **Cast** expression converts something into TimestampType, it should be resolved with setting `timezoneId`. In general, it is resolved in LogicalPlan phase.

However, there are still some places that use Cast expression statically to convert datatypes without setting `timezoneId`. In such cases,  `NoSuchElementException: None.get` will be thrown for TimestampType.

This PR is proposed to fix the issue. We have checked the whole project and found two such usages(i.e., in`TableReader` and `HiveTableScanExec`).

## How was this patch tested?

unit test

Author: donnyzone <wellfengzhu@gmail.com>

Closes #18960 from DonnyZone/spark-21739.

(cherry picked from commit 310454b)
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
@asfgit asfgit closed this in 310454b Aug 18, 2017
MatthewRBruce pushed a commit to Shopify/spark that referenced this pull request Jul 31, 2018
…it is called statically to convert something into TimestampType

## What changes were proposed in this pull request?

https://issues.apache.org/jira/projects/SPARK/issues/SPARK-21739

This issue is caused by introducing TimeZoneAwareExpression.
When the **Cast** expression converts something into TimestampType, it should be resolved with setting `timezoneId`. In general, it is resolved in LogicalPlan phase.

However, there are still some places that use Cast expression statically to convert datatypes without setting `timezoneId`. In such cases,  `NoSuchElementException: None.get` will be thrown for TimestampType.

This PR is proposed to fix the issue. We have checked the whole project and found two such usages(i.e., in`TableReader` and `HiveTableScanExec`).

## How was this patch tested?

unit test

Author: donnyzone <wellfengzhu@gmail.com>

Closes apache#18960 from DonnyZone/spark-21739.

(cherry picked from commit 310454b)
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants