[jira] [Comment Edited] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457893#comment-16457893 ] Hyukjin Kwon edited comment on SPARK-23715 at 4/29/18 3:31 AM: --- I already built Hive from source. I did this against branch-3 since it will be released soon if I followed up the discussion in Apache Hive correctly. I double checked against master - [https://github.com/apache/hive/commit/8349dbde55f479167e43cfd1f089e131d4271e5b] for this time: {code:java} hive> select from_utc_timestamp('2000-10-10 00:00:00', 'PST'); 2000-10-09 17:00:00 hive> select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); 2000-10-10 01:00:00 ./bin/hive --version SLF4J: Class path contains multiple SLF4J bindings. SLF4J: Found binding in [jar:file:/.../apache-hive-3.1.0-SNAPSHOT-bin/lib/log4j-slf4j-impl-2.10.0.jar!/org/slf4j/impl/StaticLoggerBinder.class] SLF4J: Found binding in [jar:file:/.../hadoop-2.7.5/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class] SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation. SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory] Hive 3.1.0-SNAPSHOT Git git://.../hive -r 8349dbde55f479167e43cfd1f089e131d4271e5b Compiled by hkwon on Sun Apr 29 10:55:51 +08 2018 >From source with checksum 79b284fcc099c917cc1aba844ab65e6f {code} was (Author: hyukjin.kwon): I already built Hive from source. I did this against branch-3 since it will be released soon if I followed up the discussion in Apache Hive correctly. I double checked against master - https://github.com/apache/hive/commit/8349dbde55f479167e43cfd1f089e131d4271e5b for this time: {code} hive> select from_utc_timestamp('2000-10-10 00:00:00', 'PST'); 2000-10-09 17:00:00 hive> select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); 2000-10-10 01:00:00 ./bin/hive --version SLF4J: Class path contains multiple SLF4J bindings. SLF4J: Found binding in [jar:file:/Users/hkwon/workspace/repos/forked/hive/packaging/target/apache-hive-3.1.0-SNAPSHOT-bin/apache-hive-3.1.0-SNAPSHOT-bin/lib/log4j-slf4j-impl-2.10.0.jar!/org/slf4j/impl/StaticLoggerBinder.class] SLF4J: Found binding in [jar:file:/Users/hkwon/tools/hadoop-2.7.5/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class] SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation. SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory] Hive 3.1.0-SNAPSHOT Git git://HW15435.local/Users/hkwon/workspace/repos/forked/hive -r 8349dbde55f479167e43cfd1f089e131d4271e5b Compiled by hkwon on Sun Apr 29 10:55:51 +08 2018 >From source with checksum 79b284fcc099c917cc1aba844ab65e6f {code} > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it
[jira] [Resolved] (SPARK-23724) Custom record separator for jsons in charsets different from UTF-8
[ https://issues.apache.org/jira/browse/SPARK-23724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hyukjin Kwon resolved SPARK-23724. -- Resolution: Fixed Fix Version/s: 2.4.0 Issue resolved by pull request 20937 [https://github.com/apache/spark/pull/20937] > Custom record separator for jsons in charsets different from UTF-8 > -- > > Key: SPARK-23724 > URL: https://issues.apache.org/jira/browse/SPARK-23724 > Project: Spark > Issue Type: Sub-task > Components: SQL >Affects Versions: 2.4.0 >Reporter: Maxim Gekk >Assignee: Maxim Gekk >Priority: Major > Fix For: 2.4.0 > > > The option should define a sequence of bytes between two consecutive json > records. Currently the separator is detected automatically by hadoop library: > > [https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java#L185-L254] > > The method is able to recognize only *\r, \n* and *\r\n* in UTF-8 encoding. > It doesn't work in the cases if encoding of input stream is different from > UTF-8. The option should allow to users explicitly set separator/delimiter of > json records. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Assigned] (SPARK-23723) New charset option for json datasource
[ https://issues.apache.org/jira/browse/SPARK-23723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hyukjin Kwon reassigned SPARK-23723: Assignee: Maxim Gekk > New charset option for json datasource > -- > > Key: SPARK-23723 > URL: https://issues.apache.org/jira/browse/SPARK-23723 > Project: Spark > Issue Type: Sub-task > Components: SQL >Affects Versions: 2.4.0 >Reporter: Maxim Gekk >Assignee: Maxim Gekk >Priority: Major > Fix For: 2.4.0 > > > Currently JSON Reader can read json files in different charset/encodings. The > JSON Reader uses the jackson-json library to automatically detect the charset > of input text/stream. Here you can see the method which detects encoding: > [https://github.com/FasterXML/jackson-core/blob/master/src/main/java/com/fasterxml/jackson/core/json/ByteSourceJsonBootstrapper.java#L111-L174] > > The detectEncoding method checks the BOM > ([https://en.wikipedia.org/wiki/Byte_order_mark]) at the beginning of a text. > The BOM can be in the file but it is not mandatory. If it is not present, the > auto detection mechanism can select wrong charset. And as a consequence of > that, the user cannot read the json file. *The proposed option will allow to > bypass the auto detection mechanism and set the charset explicitly.* > > The charset option is already exposed as a CSV option: > [https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala#L87-L88] > . I propose to add the same option for JSON. > > Regarding to JSON Writer, *the charset option will give to the user > opportunity* to read json files in charset different from UTF-8, modify the > dataset and *write results back to json files in the original encoding.* At > the moment it is not possible to do because the result can be saved in UTF-8 > only. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Assigned] (SPARK-23094) Json Readers choose wrong encoding when bad records are present and fail
[ https://issues.apache.org/jira/browse/SPARK-23094?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hyukjin Kwon reassigned SPARK-23094: Assignee: Maxim Gekk (was: Burak Yavuz) > Json Readers choose wrong encoding when bad records are present and fail > > > Key: SPARK-23094 > URL: https://issues.apache.org/jira/browse/SPARK-23094 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.2.1 >Reporter: Burak Yavuz >Assignee: Maxim Gekk >Priority: Major > Fix For: 2.4.0 > > > The cases described in SPARK-16548 and SPARK-20549 handled the JsonParser > code paths for expressions but not the readers. We should also cover reader > code paths reading files with bad characters. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Resolved] (SPARK-23723) New charset option for json datasource
[ https://issues.apache.org/jira/browse/SPARK-23723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hyukjin Kwon resolved SPARK-23723. -- Resolution: Fixed Fix Version/s: 2.4.0 Issue resolved by pull request 20937 [https://github.com/apache/spark/pull/20937] > New charset option for json datasource > -- > > Key: SPARK-23723 > URL: https://issues.apache.org/jira/browse/SPARK-23723 > Project: Spark > Issue Type: Sub-task > Components: SQL >Affects Versions: 2.4.0 >Reporter: Maxim Gekk >Assignee: Maxim Gekk >Priority: Major > Fix For: 2.4.0 > > > Currently JSON Reader can read json files in different charset/encodings. The > JSON Reader uses the jackson-json library to automatically detect the charset > of input text/stream. Here you can see the method which detects encoding: > [https://github.com/FasterXML/jackson-core/blob/master/src/main/java/com/fasterxml/jackson/core/json/ByteSourceJsonBootstrapper.java#L111-L174] > > The detectEncoding method checks the BOM > ([https://en.wikipedia.org/wiki/Byte_order_mark]) at the beginning of a text. > The BOM can be in the file but it is not mandatory. If it is not present, the > auto detection mechanism can select wrong charset. And as a consequence of > that, the user cannot read the json file. *The proposed option will allow to > bypass the auto detection mechanism and set the charset explicitly.* > > The charset option is already exposed as a CSV option: > [https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala#L87-L88] > . I propose to add the same option for JSON. > > Regarding to JSON Writer, *the charset option will give to the user > opportunity* to read json files in charset different from UTF-8, modify the > dataset and *write results back to json files in the original encoding.* At > the moment it is not possible to do because the result can be saved in UTF-8 > only. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Assigned] (SPARK-23724) Custom record separator for jsons in charsets different from UTF-8
[ https://issues.apache.org/jira/browse/SPARK-23724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hyukjin Kwon reassigned SPARK-23724: Assignee: Maxim Gekk > Custom record separator for jsons in charsets different from UTF-8 > -- > > Key: SPARK-23724 > URL: https://issues.apache.org/jira/browse/SPARK-23724 > Project: Spark > Issue Type: Sub-task > Components: SQL >Affects Versions: 2.4.0 >Reporter: Maxim Gekk >Assignee: Maxim Gekk >Priority: Major > > The option should define a sequence of bytes between two consecutive json > records. Currently the separator is detected automatically by hadoop library: > > [https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java#L185-L254] > > The method is able to recognize only *\r, \n* and *\r\n* in UTF-8 encoding. > It doesn't work in the cases if encoding of input stream is different from > UTF-8. The option should allow to users explicitly set separator/delimiter of > json records. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Resolved] (SPARK-23094) Json Readers choose wrong encoding when bad records are present and fail
[ https://issues.apache.org/jira/browse/SPARK-23094?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hyukjin Kwon resolved SPARK-23094. -- Resolution: Fixed Fix Version/s: 2.4.0 Issue resolved by pull request 20937 [https://github.com/apache/spark/pull/20937] > Json Readers choose wrong encoding when bad records are present and fail > > > Key: SPARK-23094 > URL: https://issues.apache.org/jira/browse/SPARK-23094 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.2.1 >Reporter: Burak Yavuz >Assignee: Burak Yavuz >Priority: Major > Fix For: 2.4.0 > > > The cases described in SPARK-16548 and SPARK-20549 handled the JsonParser > code paths for expressions but not the readers. We should also cover reader > code paths reading files with bad characters. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457896#comment-16457896 ] Bruce Robbins commented on SPARK-23715: --- [~hyukjin.kwon] Yes, I also built from sources and I could reproduce there. I could not reproduce on a released version. Anyway, I just don't want Hive to introduce the same bug. > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the actual input > or the conversion, still assumes the timestamp is shifted by the loc
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457894#comment-16457894 ] Hyukjin Kwon commented on SPARK-23715: -- Right, I agree with the PR approach. I just found a small delta and wanted to double check. > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the actual input > or the conversion, still assumes the timestamp is shifted by the local time > zone. So it reverse-shifts the long value by the local time zone's offset, > w
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457893#comment-16457893 ] Hyukjin Kwon commented on SPARK-23715: -- I already built Hive from source. I did this against branch-3 since it will be released soon if I followed up the discussion in Apache Hive correctly. I double checked against master - https://github.com/apache/hive/commit/8349dbde55f479167e43cfd1f089e131d4271e5b for this time: {code} hive> select from_utc_timestamp('2000-10-10 00:00:00', 'PST'); 2000-10-09 17:00:00 hive> select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); 2000-10-10 01:00:00 ./bin/hive --version SLF4J: Class path contains multiple SLF4J bindings. SLF4J: Found binding in [jar:file:/Users/hkwon/workspace/repos/forked/hive/packaging/target/apache-hive-3.1.0-SNAPSHOT-bin/apache-hive-3.1.0-SNAPSHOT-bin/lib/log4j-slf4j-impl-2.10.0.jar!/org/slf4j/impl/StaticLoggerBinder.class] SLF4J: Found binding in [jar:file:/Users/hkwon/tools/hadoop-2.7.5/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class] SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation. SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory] Hive 3.1.0-SNAPSHOT Git git://HW15435.local/Users/hkwon/workspace/repos/forked/hive -r 8349dbde55f479167e43cfd1f089e131d4271e5b Compiled by hkwon on Sun Apr 29 10:55:51 +08 2018 >From source with checksum 79b284fcc099c917cc1aba844ab65e6f {code} > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457892#comment-16457892 ] Bruce Robbins commented on SPARK-23715: --- Still, I filed an Jira with Hive so they won't release this bug. Currently this bug does not exist in any release available in their downloads page. https://issues.apache.org/jira/browse/HIVE-19354 > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the actual input > or the conversion, still assumes the timestamp is
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457888#comment-16457888 ] Wenchen Fan commented on SPARK-23715: - `from_utc_timestamp` is also in some traditional RDBMS like DB2. From the DB2 doc: https://www.ibm.com/support/knowledgecenter/en/SS6NHC/com.ibm.swg.im.dashdb.sql.ref.doc/doc/r0061916.html. > An expression that specifies the timestamp that is in the Coordinated > Universal Time time zone. I think this is a reasonable definition and we should reject timestamp string that contains timezone. We can look at more databases and follow the mainstream. > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimesta
[jira] [Commented] (SPARK-22938) Assert that SQLConf.get is accessed only on the driver.
[ https://issues.apache.org/jira/browse/SPARK-22938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457879#comment-16457879 ] Apache Spark commented on SPARK-22938: -- User 'cloud-fan' has created a pull request for this issue: https://github.com/apache/spark/pull/21190 > Assert that SQLConf.get is accessed only on the driver. > --- > > Key: SPARK-22938 > URL: https://issues.apache.org/jira/browse/SPARK-22938 > Project: Spark > Issue Type: Test > Components: SQL >Affects Versions: 2.2.1 >Reporter: Juliusz Sompolski >Assignee: Juliusz Sompolski >Priority: Major > Fix For: 2.3.0 > > > Assert if code tries to access SQLConf.get on executor. > This can lead to hard to detect bugs, where the executor will read > fallbackConf, falling back to default config values, ignoring potentially > changed non-default configs. > If a config is to be passed to executor code, it needs to be read on the > driver, and passed explicitly. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457874#comment-16457874 ] Bruce Robbins commented on SPARK-23715: --- I might understand what's going on with Hive. In the latest released version of Hive (2..3), their timestamp converter code uses java.sql.Timestamp to convert string inputs, which expects JDBC escaped format (-mm-dd hh:mm:ss.f) [https://github.com/apache/hive/blob/branch-2.3/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java#L1181] In the Hive master branch however, it looks like they added some code to try to parse a string value containing a timezone, if java.sql.Timestamp.valueOf fails: [https://github.com/apache/hive/blob/master/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java#L1250] [~hyukjin.kwon] Is it possible you tested on the master branch? If this is the explanation, then Hive does not share Spark's bug _yet_, but they will soon. > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is r
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457754#comment-16457754 ] Bruce Robbins commented on SPARK-23715: --- I just downloaded and installed hive-2.3.3 (3 April 2018) from Apache and ran it in standalone mode. I still see the behavior I earlier reported: {noformat} hive> select from_utc_timestamp('2000-10-10 00:00:00', 'PST'); OK 2000-10-09 17:00:00 Time taken: 0.016 seconds, Fetched: 1 row(s) hive> select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); OK NULL Time taken: 0.015 seconds, Fetched: 1 row(s) hive> $ hive --version Hive 2.3.3 Git git://daijymacpro-2.local/Users/daijy/commit/hive -r 8a511e3f79b43d4be41cd231cf5c99e43b248383 Compiled by daijy on Wed Mar 28 16:58:33 PDT 2018 >From source with checksum 8873bba6c55a058614e74c0e628ab022 {noformat} Maybe you are running Hive-on-Spark? > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457674#comment-16457674 ] Bruce Robbins commented on SPARK-23715: --- Could be this: HIVE-14412 > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the actual input > or the conversion, still assumes the timestamp is shifted by the local time > zone. So it reverse-shifts the long value by the local time zone's offset, > which produces a incorrect timestamp (except in the case where the
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457669#comment-16457669 ] Hyukjin Kwon commented on SPARK-23715: -- Yea but was just wondering what behaviour Hive wants. It should be good to read the discussion if there's a JIRA for it. Timezone stuff always makes me think multiple times for a while. FYI, my timezone is Singapore Timezone. > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the actual input > or the conversion, st
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457665#comment-16457665 ] Bruce Robbins commented on SPARK-23715: --- {quote}Which version did you use?{quote} The jars all say "2.1.1". I can check about datetime string formats with some Hive experts on Monday. By the way, that Hive result is still not correct (but might be consistent with Spark, depending on your timezone). > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the ac
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457658#comment-16457658 ] Hyukjin Kwon commented on SPARK-23715: -- I used the (almost) latest build one. Will double check and track the JIRA tomorrow (it's quite late in my time). Which version did you use? > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > offset, but by the timezone specified on the datetime string. > Unfortunately, FromUTCTimestamp, which has no insight into the actual input > or the conversion, still assumes the timestamp is shifted by the local time > zone. So it reverse-shifts the
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457657#comment-16457657 ] Bruce Robbins commented on SPARK-23715: --- Maybe a configuration setting or difference between versions? {noformat} hive> select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); OK NULL {noformat} Also the docs say: Supported conversions: - Strings: JDBC compliant java.sql.Timestamp format "-MM-DD HH:MM:SS.f" (9 decimal place precision) I am not a Hive user, so I am going by the document and the above test. > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the local timezone's > o
[jira] [Updated] (SPARK-23171) Reduce the time costs of the rule runs that do not change the plans
[ https://issues.apache.org/jira/browse/SPARK-23171?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Teng Peng updated SPARK-23171: -- Issue Type: Improvement (was: Umbrella) > Reduce the time costs of the rule runs that do not change the plans > > > Key: SPARK-23171 > URL: https://issues.apache.org/jira/browse/SPARK-23171 > Project: Spark > Issue Type: Improvement > Components: SQL >Affects Versions: 2.3.0 >Reporter: Xiao Li >Priority: Major > > Below is the time stats of Analyzer/Optimizer rules. Try to improve the rules > and reduce the time costs, especially for the runs that do not change the > plans. > {noformat} > === Metrics of Analyzer/Optimizer Rules === > Total number of runs = 175827 > Total time: 20.699042877 seconds > Rule > Total Time Effective Time Total Runs > Effective Runs > org.apache.spark.sql.catalyst.optimizer.ColumnPruning > 2340563794 1338268224 1875 > 761 > org.apache.spark.sql.catalyst.analysis.Analyzer$CTESubstitution > 1632672623 1625071881 788 > 37 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions > 1395087131 347339931 1982 > 38 > org.apache.spark.sql.catalyst.optimizer.PruneFilters > 1177711364 21344174 1590 > 3 > org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries > 1145135465 1131417128 285 > 39 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences > 1008347217 663112062 1982 > 616 > org.apache.spark.sql.catalyst.optimizer.ReorderJoin > 767024424 693001699 1590 > 132 > org.apache.spark.sql.catalyst.analysis.Analyzer$FixNullability > 598524650 40802876 742 > 12 > org.apache.spark.sql.catalyst.analysis.DecimalPrecision > 595384169 436153128 1982 > 211 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubquery > 548178270 459695885 1982 > 49 > org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts > 423002864 139869503 1982 > 86 > org.apache.spark.sql.catalyst.optimizer.BooleanSimplification > 405544962 17250184 1590 > 7 > org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughJoin > 383837603 284174662 1590 > 708 > org.apache.spark.sql.catalyst.optimizer.RemoveRedundantAliases > 372901885 33623321590 > 9 > org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints > 364628214 343815519 285 > 192 > org.apache.spark.sql.execution.datasources.FindDataSourceTable > 303293296 285344766 1982 > 233 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions > 233195019 92648171 1982 > 294 > org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion > 220568919 73932736 1982 > 38 > org.apache.spark.sql.catalyst.optimizer.NullPropagation > 207976072 90723051590 > 26 > org.apache.spark.sql.catalyst.analysis.TypeCoercion$P
[jira] [Updated] (SPARK-23171) Reduce the time costs of the rule runs that do not change the plans
[ https://issues.apache.org/jira/browse/SPARK-23171?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Teng Peng updated SPARK-23171: -- Issue Type: Umbrella (was: Improvement) > Reduce the time costs of the rule runs that do not change the plans > > > Key: SPARK-23171 > URL: https://issues.apache.org/jira/browse/SPARK-23171 > Project: Spark > Issue Type: Umbrella > Components: SQL >Affects Versions: 2.3.0 >Reporter: Xiao Li >Priority: Major > > Below is the time stats of Analyzer/Optimizer rules. Try to improve the rules > and reduce the time costs, especially for the runs that do not change the > plans. > {noformat} > === Metrics of Analyzer/Optimizer Rules === > Total number of runs = 175827 > Total time: 20.699042877 seconds > Rule > Total Time Effective Time Total Runs > Effective Runs > org.apache.spark.sql.catalyst.optimizer.ColumnPruning > 2340563794 1338268224 1875 > 761 > org.apache.spark.sql.catalyst.analysis.Analyzer$CTESubstitution > 1632672623 1625071881 788 > 37 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions > 1395087131 347339931 1982 > 38 > org.apache.spark.sql.catalyst.optimizer.PruneFilters > 1177711364 21344174 1590 > 3 > org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries > 1145135465 1131417128 285 > 39 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences > 1008347217 663112062 1982 > 616 > org.apache.spark.sql.catalyst.optimizer.ReorderJoin > 767024424 693001699 1590 > 132 > org.apache.spark.sql.catalyst.analysis.Analyzer$FixNullability > 598524650 40802876 742 > 12 > org.apache.spark.sql.catalyst.analysis.DecimalPrecision > 595384169 436153128 1982 > 211 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubquery > 548178270 459695885 1982 > 49 > org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts > 423002864 139869503 1982 > 86 > org.apache.spark.sql.catalyst.optimizer.BooleanSimplification > 405544962 17250184 1590 > 7 > org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughJoin > 383837603 284174662 1590 > 708 > org.apache.spark.sql.catalyst.optimizer.RemoveRedundantAliases > 372901885 33623321590 > 9 > org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints > 364628214 343815519 285 > 192 > org.apache.spark.sql.execution.datasources.FindDataSourceTable > 303293296 285344766 1982 > 233 > org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions > 233195019 92648171 1982 > 294 > org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion > 220568919 73932736 1982 > 38 > org.apache.spark.sql.catalyst.optimizer.NullPropagation > 207976072 90723051590 > 26 > org.apache.spark.sql.catalyst.analysis.TypeCoercion$Prom
[jira] [Commented] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457604#comment-16457604 ] Hyukjin Kwon commented on SPARK-23715: -- [~bersprockets], mind if I ask to clarify {quote} In Hive and Impala, the user can only enter a datetime value in the format "yyy-MM-dd HH:mm". With Hive and Impala, the user cannot enter a timezone specification. {quote} ? I was double checking it but seems Hive accepts timezone part in the input: {code} hive> select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); 2000-10-10 01:00:00 {code} > from_utc_timestamp returns incorrect results for some UTC date/time values > -- > > Key: SPARK-23715 > URL: https://issues.apache.org/jira/browse/SPARK-23715 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.3.0 >Reporter: Bruce Robbins >Priority: Major > > This produces the expected answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 07:18:23| > +---+ > {noformat} > However, the equivalent UTC input (but with an explicit timezone) produces a > wrong answer: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > Additionally, the equivalent Unix time (1520921903, which is also > "2018-03-13T06:18:23" in the UTC time zone) produces the same wrong answer: > {noformat} > df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" > ).as("dt")).show > +---+ > | dt| > +---+ > |2018-03-13 00:18:23| > +---+ > {noformat} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 152094710300) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 152092190300) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (152094710300 minus 7 hours is 152092190300). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > There is sometimes a mismatch in expectations between the (string => > timestamp) cast and FromUTCTimestamp. Also, since the FromUTCTimestamp > expression never sees the actual input string (the cast "intercepts" the > input and converts it to a long timestamp before FromUTCTimestamp uses the > value), FromUTCTimestamp cannot reject any input value that would exercise > this mismatch in expectations. > There is a similar mismatch in expectations in the (integer => timestamp) > cast and FromUTCTimestamp. As a result, Unix time input almost always > produces incorrect output. > h3. When things work as expected for String input: > When from_utc_timestamp is passed a string time value with no time zone, > DateTimeUtils.stringToTimestamp (called from a Cast expression) treats the > datetime string as though it's in the user's local time zone. Because > DateTimeUtils.stringToTimestamp is a general function, this is reasonable. > As a result, FromUTCTimestamp's input is a timestamp shifted by the local > time zone's offset. FromUTCTimestamp assumes this (or more accurately, a > utility function called by FromUTCTimestamp assumes this), so the first thing > it does is reverse-shift to get it back the correct value. Now that the long > value has been shifted back to the correct timestamp value, it can now > process it (by shifting it again based on the specified time zone). > h3. When things go wrong with String input: > When from_utc_timestamp is passed a string datetime value with an explicit > time zone, stringToTimestamp honors that timezone and ignores the local time > zone. stringToTimestamp does not shift the timestamp by the loca
[jira] [Commented] (SPARK-24117) Unified the getSizePerRow
[ https://issues.apache.org/jira/browse/SPARK-24117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457549#comment-16457549 ] Apache Spark commented on SPARK-24117: -- User 'wangyum' has created a pull request for this issue: https://github.com/apache/spark/pull/21189 > Unified the getSizePerRow > - > > Key: SPARK-24117 > URL: https://issues.apache.org/jira/browse/SPARK-24117 > Project: Spark > Issue Type: Improvement > Components: SQL >Affects Versions: 2.4.0 >Reporter: Yuming Wang >Priority: Major > -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Assigned] (SPARK-24117) Unified the getSizePerRow
[ https://issues.apache.org/jira/browse/SPARK-24117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apache Spark reassigned SPARK-24117: Assignee: (was: Apache Spark) > Unified the getSizePerRow > - > > Key: SPARK-24117 > URL: https://issues.apache.org/jira/browse/SPARK-24117 > Project: Spark > Issue Type: Improvement > Components: SQL >Affects Versions: 2.4.0 >Reporter: Yuming Wang >Priority: Major > -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Assigned] (SPARK-24117) Unified the getSizePerRow
[ https://issues.apache.org/jira/browse/SPARK-24117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apache Spark reassigned SPARK-24117: Assignee: Apache Spark > Unified the getSizePerRow > - > > Key: SPARK-24117 > URL: https://issues.apache.org/jira/browse/SPARK-24117 > Project: Spark > Issue Type: Improvement > Components: SQL >Affects Versions: 2.4.0 >Reporter: Yuming Wang >Assignee: Apache Spark >Priority: Major > -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-24117) Unified the getSizePerRow
Yuming Wang created SPARK-24117: --- Summary: Unified the getSizePerRow Key: SPARK-24117 URL: https://issues.apache.org/jira/browse/SPARK-24117 Project: Spark Issue Type: Improvement Components: SQL Affects Versions: 2.4.0 Reporter: Yuming Wang -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-24046) Rate Source doesn't gradually increase rate when rampUpTime>=RowsPerSecond
[ https://issues.apache.org/jira/browse/SPARK-24046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16457430#comment-16457430 ] Apache Spark commented on SPARK-24046: -- User 'jerryshao' has created a pull request for this issue: https://github.com/apache/spark/pull/21188 > Rate Source doesn't gradually increase rate when rampUpTime>=RowsPerSecond > -- > > Key: SPARK-24046 > URL: https://issues.apache.org/jira/browse/SPARK-24046 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 > Environment: Spark 2.3.0 using Spark Shell on Ubuntu 17.4 > (Environment is not important, the issue lies in the rate calculation) >Reporter: Gerard Maas >Priority: Major > Labels: RateSource > Attachments: image-2018-04-22-22-03-03-945.png, > image-2018-04-22-22-06-49-202.png > > > When using the rate source in Structured streaming, the `rampUpTime` feature > fails to gradually increase the stream rate when the `rampUpTime` option is > equal or greater than `rowsPerSecond`. > When rampUpTime >= rowsPerSecond` all batches at `time < rampUpTime` contain > 0 values. The rate jumps to `rowsPerSecond` when `time>rampUpTime`. > The following scenario, executed in the `spark-shell` demonstrates this issue: > {code:java} > // Using rampUpTime(10) > rowsPerSecond(5) > {code} > {code:java} > val stream = spark.readStream > .format("rate") > .option("rowsPerSecond", 5) > .option("rampUpTime", 10) > .load() > val query = stream.writeStream.format("console").start() > // Exiting paste mode, now interpreting. > stream: org.apache.spark.sql.DataFrame = [timestamp: timestamp, value: bigint] > query: org.apache.spark.sql.streaming.StreamingQuery = > org.apache.spark.sql.execution.streaming.StreamingQueryWrapper@cf82c58 > --- > Batch: 0 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 1 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 2 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 3 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 4 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 5 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 6 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 7 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 8 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 9 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 10 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 11 > --- > ++-+ > | timestamp|value| > ++-+ > |2018-04-22 17:08:...| 0| > |2018-04-22 17:08:...| 1| > |2018-04-22 17:08:...| 2| > |2018-04-22 17:08:...| 3| > |2018-04-22 17:08:...| 4| > ++-+ > --- > Batch: 12 > --- > ++-+ > | timestamp|value| > ++-+ > |2018-04-22 17:08:...| 5| > |2018-04-22 17:08:...| 6| > |2018-04-22 17:08:...| 7| > |2018-04-22 17:08:...| 8| > |2018-04-22 17:08:...| 9| > ++-+ > {code} > > This scenario shows rowsPerSecond == rampUpTime, which also fails > {code:java} > val stream = spark.readStream > .format("rate") > .option("rowsPerSecond", 10) > .option("rampUpTime", 10) > .load() > val query = s
[jira] [Assigned] (SPARK-24046) Rate Source doesn't gradually increase rate when rampUpTime>=RowsPerSecond
[ https://issues.apache.org/jira/browse/SPARK-24046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apache Spark reassigned SPARK-24046: Assignee: (was: Apache Spark) > Rate Source doesn't gradually increase rate when rampUpTime>=RowsPerSecond > -- > > Key: SPARK-24046 > URL: https://issues.apache.org/jira/browse/SPARK-24046 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 > Environment: Spark 2.3.0 using Spark Shell on Ubuntu 17.4 > (Environment is not important, the issue lies in the rate calculation) >Reporter: Gerard Maas >Priority: Major > Labels: RateSource > Attachments: image-2018-04-22-22-03-03-945.png, > image-2018-04-22-22-06-49-202.png > > > When using the rate source in Structured streaming, the `rampUpTime` feature > fails to gradually increase the stream rate when the `rampUpTime` option is > equal or greater than `rowsPerSecond`. > When rampUpTime >= rowsPerSecond` all batches at `time < rampUpTime` contain > 0 values. The rate jumps to `rowsPerSecond` when `time>rampUpTime`. > The following scenario, executed in the `spark-shell` demonstrates this issue: > {code:java} > // Using rampUpTime(10) > rowsPerSecond(5) > {code} > {code:java} > val stream = spark.readStream > .format("rate") > .option("rowsPerSecond", 5) > .option("rampUpTime", 10) > .load() > val query = stream.writeStream.format("console").start() > // Exiting paste mode, now interpreting. > stream: org.apache.spark.sql.DataFrame = [timestamp: timestamp, value: bigint] > query: org.apache.spark.sql.streaming.StreamingQuery = > org.apache.spark.sql.execution.streaming.StreamingQueryWrapper@cf82c58 > --- > Batch: 0 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 1 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 2 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 3 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 4 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 5 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 6 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 7 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 8 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 9 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 10 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 11 > --- > ++-+ > | timestamp|value| > ++-+ > |2018-04-22 17:08:...| 0| > |2018-04-22 17:08:...| 1| > |2018-04-22 17:08:...| 2| > |2018-04-22 17:08:...| 3| > |2018-04-22 17:08:...| 4| > ++-+ > --- > Batch: 12 > --- > ++-+ > | timestamp|value| > ++-+ > |2018-04-22 17:08:...| 5| > |2018-04-22 17:08:...| 6| > |2018-04-22 17:08:...| 7| > |2018-04-22 17:08:...| 8| > |2018-04-22 17:08:...| 9| > ++-+ > {code} > > This scenario shows rowsPerSecond == rampUpTime, which also fails > {code:java} > val stream = spark.readStream > .format("rate") > .option("rowsPerSecond", 10) > .option("rampUpTime", 10) > .load() > val query = stream.writeStream.format("console").start() > // Exiting paste mode, now interpreting. > stream: org.apache.spark.sq
[jira] [Assigned] (SPARK-24046) Rate Source doesn't gradually increase rate when rampUpTime>=RowsPerSecond
[ https://issues.apache.org/jira/browse/SPARK-24046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apache Spark reassigned SPARK-24046: Assignee: Apache Spark > Rate Source doesn't gradually increase rate when rampUpTime>=RowsPerSecond > -- > > Key: SPARK-24046 > URL: https://issues.apache.org/jira/browse/SPARK-24046 > Project: Spark > Issue Type: Bug > Components: Structured Streaming >Affects Versions: 2.3.0 > Environment: Spark 2.3.0 using Spark Shell on Ubuntu 17.4 > (Environment is not important, the issue lies in the rate calculation) >Reporter: Gerard Maas >Assignee: Apache Spark >Priority: Major > Labels: RateSource > Attachments: image-2018-04-22-22-03-03-945.png, > image-2018-04-22-22-06-49-202.png > > > When using the rate source in Structured streaming, the `rampUpTime` feature > fails to gradually increase the stream rate when the `rampUpTime` option is > equal or greater than `rowsPerSecond`. > When rampUpTime >= rowsPerSecond` all batches at `time < rampUpTime` contain > 0 values. The rate jumps to `rowsPerSecond` when `time>rampUpTime`. > The following scenario, executed in the `spark-shell` demonstrates this issue: > {code:java} > // Using rampUpTime(10) > rowsPerSecond(5) > {code} > {code:java} > val stream = spark.readStream > .format("rate") > .option("rowsPerSecond", 5) > .option("rampUpTime", 10) > .load() > val query = stream.writeStream.format("console").start() > // Exiting paste mode, now interpreting. > stream: org.apache.spark.sql.DataFrame = [timestamp: timestamp, value: bigint] > query: org.apache.spark.sql.streaming.StreamingQuery = > org.apache.spark.sql.execution.streaming.StreamingQueryWrapper@cf82c58 > --- > Batch: 0 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 1 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 2 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 3 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 4 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 5 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 6 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 7 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 8 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 9 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 10 > --- > +-+-+ > |timestamp|value| > +-+-+ > +-+-+ > --- > Batch: 11 > --- > ++-+ > | timestamp|value| > ++-+ > |2018-04-22 17:08:...| 0| > |2018-04-22 17:08:...| 1| > |2018-04-22 17:08:...| 2| > |2018-04-22 17:08:...| 3| > |2018-04-22 17:08:...| 4| > ++-+ > --- > Batch: 12 > --- > ++-+ > | timestamp|value| > ++-+ > |2018-04-22 17:08:...| 5| > |2018-04-22 17:08:...| 6| > |2018-04-22 17:08:...| 7| > |2018-04-22 17:08:...| 8| > |2018-04-22 17:08:...| 9| > ++-+ > {code} > > This scenario shows rowsPerSecond == rampUpTime, which also fails > {code:java} > val stream = spark.readStream > .format("rate") > .option("rowsPerSecond", 10) > .option("rampUpTime", 10) > .load() > val query = stream.writeStream.format("console").start() > // Exiting paste mode, now interpreting. > st