[ 
https://issues.apache.org/jira/browse/SPARK-20357?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jeeyoung Kim updated SPARK-20357:
---------------------------------
    Description: 
Since weeks and years are extracted using different boundaries (weeks happen 
every 7 days, years happen every 365-ish days, which is not divisible by 7), 
there are weird inconsistencies around how end-of-the year dates are handled if 
you use {{year}} and {{weekofyear}} Spark SQL functions. The example below 
shows how "2016-01-01" and "2016-12-30" has the same {{(year, week)}} pair.

This happens because the week for "2016-01-01" is calculated as "last week of 
2015". the Year function in Spark SQL ignores this and returns YYYY component 
of YYYY-MM-DD.

The correct way to fix this is by exposing {{Java.util.dates.getWeekYear}}. 
This function calculates week-based years, so "2016-01-01" will return 2015 
instead. in this case.

{noformat}
# Trying out the bug for date - using PySpark
import pyspark.sql.functions as F
df = spark.createDataFrame([("2016-12-31",),("2016-12-30",), ("2017-01-01",), 
("2017-01-02",),("2017-12-30",)], ['id'])
df_parsed = (
    df
    .withColumn("year", F.year(df['id'].cast("date")))
    .withColumn("weekofyear", F.weekofyear(df['id'].cast("date")))
)
df_parsed.show()
{noformat}

Prints 
{noformat}
+----------+----+----------+
|        id|year|weekofyear|
+----------+----+----------+
|2016-12-31|2016|        52|
|2016-12-30|2016|        52|
|2017-01-01|2017|        52| <- same (year, weekofyear) output
|2017-01-02|2017|         1|
|2017-12-30|2017|        52| <- 
+----------+----+----------+
{noformat}


  was:
Since weeks and years are extracted using different boundaries (weeks happen 
every 7 days, years happen every 365-ish days, which is not divisible by 7), 
there are weird inconsistencies around how end-of-the year dates are handled if 
you use {{year}} and {{weekofyear}} Spark SQL functions. The example below 
shows how "2016-01-01" and "2016-12-30" has the same {{(year, week)}} pair.

This happens because the week for "2016-01-01" is calculated as "last week of 
2015". the Year function in Spark SQL ignores this and returns YYYY component 
of YYYY-MM-DD.

The correct way to fix this is by exposing {{Java.util.dates.getWeekYear}}. 
This function calculates week-based years, so "2016-01-01" will return 2015 
instead. in this case.

{noformat}
# Trying out the bug for date - using PySpark
import pyspark.sql.functions as F
df = spark.createDataFrame([("2016-12-31",),("2016-12-30",), ("2017-01-01",), 
("2017-01-02",),("2017-12-30",)], ['id'])
df_parsed = (
    df
    .withColumn("year", F.year(df['id'].cast("date")))
    .withColumn("weekofyear", F.weekofyear(df['id'].cast("date")))
)
df_parsed.show()
{noformat}

Prints 
{noformat}
+----------+----+----------+
|        id|year|weekofyear|
+----------+----+----------+
|2016-12-31|2016|        52|
|2016-12-30|2016|        52|
|2017-01-01|2017|        52|
|2017-01-02|2017|         1| <- same (year, weekofyear) output
|2017-12-30|2017|        52| <- 
+----------+----+----------+
{noformat}



> Expose Calendar.getWeekYear() as Spark SQL date function to be consistent 
> with weekofyear()
> -------------------------------------------------------------------------------------------
>
>                 Key: SPARK-20357
>                 URL: https://issues.apache.org/jira/browse/SPARK-20357
>             Project: Spark
>          Issue Type: New Feature
>          Components: SQL
>    Affects Versions: 2.1.0, 2.2.0
>            Reporter: Jeeyoung Kim
>            Priority: Minor
>
> Since weeks and years are extracted using different boundaries (weeks happen 
> every 7 days, years happen every 365-ish days, which is not divisible by 7), 
> there are weird inconsistencies around how end-of-the year dates are handled 
> if you use {{year}} and {{weekofyear}} Spark SQL functions. The example below 
> shows how "2016-01-01" and "2016-12-30" has the same {{(year, week)}} pair.
> This happens because the week for "2016-01-01" is calculated as "last week of 
> 2015". the Year function in Spark SQL ignores this and returns YYYY component 
> of YYYY-MM-DD.
> The correct way to fix this is by exposing {{Java.util.dates.getWeekYear}}. 
> This function calculates week-based years, so "2016-01-01" will return 2015 
> instead. in this case.
> {noformat}
> # Trying out the bug for date - using PySpark
> import pyspark.sql.functions as F
> df = spark.createDataFrame([("2016-12-31",),("2016-12-30",), ("2017-01-01",), 
> ("2017-01-02",),("2017-12-30",)], ['id'])
> df_parsed = (
>     df
>     .withColumn("year", F.year(df['id'].cast("date")))
>     .withColumn("weekofyear", F.weekofyear(df['id'].cast("date")))
> )
> df_parsed.show()
> {noformat}
> Prints 
> {noformat}
> +----------+----+----------+
> |        id|year|weekofyear|
> +----------+----+----------+
> |2016-12-31|2016|        52|
> |2016-12-30|2016|        52|
> |2017-01-01|2017|        52| <- same (year, weekofyear) output
> |2017-01-02|2017|         1|
> |2017-12-30|2017|        52| <- 
> +----------+----+----------+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to