Re: [Spark SQL] Nanoseconds in Timestamps are set as Microseconds

2017-06-01 Thread Reynold Xin
Seems like a bug we should fix? I agree some form of truncation makes more
sense.


On Thu, Jun 1, 2017 at 1:17 AM, Anton Okolnychyi  wrote:

> Hi all,
>
> I would like to ask what the community thinks regarding the way how Spark
> handles nanoseconds in the Timestamp type.
>
> As far as I see in the code, Spark assumes microseconds precision.
> Therefore, I expect to have a truncated to microseconds timestamp or an
> exception if I specify a timestamp with nanoseconds. However, the current
> implementation just silently sets nanoseconds as microseconds in [1], which
> results in a wrong timestamp. Consider the example below:
>
> spark.sql("SELECT cast('2015-01-02 00:00:00.1' as
> TIMESTAMP)").show(false)
> ++
> |CAST(2015-01-02 00:00:00.1 AS TIMESTAMP)|
> ++
> |2015-01-02 00:00:00.01  |
> ++
>
> This issue was already raised in SPARK-17914 but I do not see any decision
> there.
>
> [1] - org.apache.spark.sql.catalyst.util.DateTimeUtils, toJavaTimestamp,
> line 204
>
> Best regards,
> Anton
>


Re: [VOTE] Apache Spark 2.2.0 (RC2)

2017-06-01 Thread Reynold Xin
Again (I've probably said this more than 10 times already in different
threads), SPARK-18350 has no impact on whether the timestamp type is with
timezone or without timezone. It simply allows a session specific timezone
setting rather than having Spark always rely on the machine timezone.

On Wed, May 31, 2017 at 11:58 AM, Kostas Sakellis 
wrote:

> Hey Michael,
>
> There is a discussion on TIMESTAMP semantics going on the thread "SQL
> TIMESTAMP semantics vs. SPARK-18350" which might impact Spark 2.2. Should
> we make a decision there before voting on the next RC for Spark 2.2?
>
> Thanks,
> Kostas
>
> On Tue, May 30, 2017 at 12:09 PM, Michael Armbrust  > wrote:
>
>> Last call, anything else important in-flight for 2.2?
>>
>> On Thu, May 25, 2017 at 10:56 AM, Michael Allman 
>> wrote:
>>
>>> PR is here: https://github.com/apache/spark/pull/18112
>>>
>>>
>>> On May 25, 2017, at 10:28 AM, Michael Allman 
>>> wrote:
>>>
>>> Michael,
>>>
>>> If you haven't started cutting the new RC, I'm working on a
>>> documentation PR right now I'm hoping we can get into Spark 2.2 as a
>>> migration note, even if it's just a mention: https://issues.apache
>>> .org/jira/browse/SPARK-20888.
>>>
>>> Michael
>>>
>>>
>>> On May 22, 2017, at 11:39 AM, Michael Armbrust 
>>> wrote:
>>>
>>> I'm waiting for SPARK-20814
>>>  at Marcelo's
>>> request and I'd also like to include SPARK-20844
>>> .  I think we should
>>> be able to cut another RC midweek.
>>>
>>> On Fri, May 19, 2017 at 11:53 AM, Nick Pentreath <
>>> nick.pentre...@gmail.com> wrote:
>>>
 All the outstanding ML QA doc and user guide items are done for 2.2 so
 from that side we should be good to cut another RC :)


 On Thu, 18 May 2017 at 00:18 Russell Spitzer 
 wrote:

> Seeing an issue with the DataScanExec and some of our integration
> tests for the SCC. Running dataframe read and writes from the shell seems
> fine but the Redaction code seems to get a "None" when doing
> SparkSession.getActiveSession.get in our integration tests. I'm not
> sure why but i'll dig into this later if I get a chance.
>
> Example Failed Test
> https://github.com/datastax/spark-cassandra-connector/blob/v
> 2.0.1/spark-cassandra-connector/src/it/scala/com/datastax/sp
> ark/connector/sql/CassandraSQLSpec.scala#L311
>
> ```[info]   org.apache.spark.SparkException: Job aborted due to stage
> failure: Task serialization failed: java.util.NoSuchElementException:
> None.get
> [info] java.util.NoSuchElementException: None.get
> [info] at scala.None$.get(Option.scala:347)
> [info] at scala.None$.get(Option.scala:345)
> [info] at org.apache.spark.sql.execution.DataSourceScanExec$class.org$
> apache$spark$sql$execution$DataSourceScanExec$$redact(DataSo
> urceScanExec.scala:70)
> [info] at org.apache.spark.sql.execution.DataSourceScanExec$$anonfun$4
> .apply(DataSourceScanExec.scala:54)
> [info] at org.apache.spark.sql.execution.DataSourceScanExec$$anonfun$4
> .apply(DataSourceScanExec.scala:52)
> ```
>
> Again this only seems to repo in our IT suite so i'm not sure if this
> is a real issue.
>
>
> On Tue, May 16, 2017 at 1:40 PM Joseph Bradley 
> wrote:
>
>> All of the ML/Graph/SparkR QA blocker JIRAs have been resolved.
>> Thanks everyone who helped out on those!
>>
>> We still have open ML/Graph/SparkR JIRAs targeted at 2.2, but they
>> are essentially all for documentation.
>>
>> Joseph
>>
>> On Thu, May 11, 2017 at 3:08 PM, Marcelo Vanzin 
>> wrote:
>>
>>> Since you'll be creating a new RC, I'd wait until SPARK-20666 is
>>> fixed, since the change that caused it is in branch-2.2. Probably a
>>> good idea to raise it to blocker at this point.
>>>
>>> On Thu, May 11, 2017 at 2:59 PM, Michael Armbrust
>>>  wrote:
>>> > I'm going to -1 given the outstanding issues and lack of +1s.
>>> I'll create
>>> > another RC once ML has had time to take care of the more critical
>>> problems.
>>> > In the meantime please keep testing this release!
>>> >
>>> > On Tue, May 9, 2017 at 2:00 AM, Kazuaki Ishizaki <
>>> ishiz...@jp.ibm.com>
>>> > wrote:
>>> >>
>>> >> +1 (non-binding)
>>> >>
>>> >> I tested it on Ubuntu 16.04 and OpenJDK8 on ppc64le. All of the
>>> tests for
>>> >> core have passed.
>>> >>
>>> >> $ java -version
>>> >> openjdk version "1.8.0_111"
>>> >> OpenJDK Runtime Environment (build
>>> >> 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14)
>>> >> OpenJDK 64-Bit Server VM (build 25.111-b14, mixed mode)
>>> >> $ build/mvn -DskipTests -Phive -Phive-thriftserver -Pyarn
>>> -Phadoop-2.7
>>> >> package install
>>> >> $ build/mvn -Phive -Phive-thriftserver -Pyarn -Phadoop-2.7 test
>>> -pl c

Re: SQL TIMESTAMP semantics vs. SPARK-18350

2017-06-01 Thread Reynold Xin
Yea I don't see why this needs to be per table config. If the user wants to
configure it per table, can't they just declare the data type on a per
table basis, once we have separate types for timestamp w/ tz and w/o tz?

On Thu, Jun 1, 2017 at 4:14 PM, Michael Allman  wrote:

> I would suggest that making timestamp type behavior configurable and
> persisted per-table could introduce some real confusion, e.g. in queries
> involving tables with different timestamp type semantics.
>
> I suggest starting with the assumption that timestamp type behavior is a
> per-session flag that can be set in a global `spark-defaults.conf` and
> consider more granular levels of configuration as people identify solid use
> cases.
>
> Cheers,
>
> Michael
>
>
>
> On May 30, 2017, at 7:41 AM, Zoltan Ivanfi  wrote:
>
> Hi,
>
> If I remember correctly, the TIMESTAMP type had UTC-normalized local time
> semantics even before Spark 2, so I can understand that Spark considers it
> to be the "established" behavior that must not be broken. Unfortunately,
> this behavior does not provide interoperability with other SQL engines of
> the Hadoop stack.
>
> Let me summarize the findings of this e-mail thread so far:
>
>- Timezone-agnostic TIMESTAMP semantics would be beneficial for
>interoperability and SQL compliance.
>- Spark can not make a breaking change. For backward-compatibility
>with existing data, timestamp semantics should be user-configurable on a
>per-table level.
>
> Before going into the specifics of a possible solution, do we all agree on
> these points?
>
> Thanks,
>
> Zoltan
>
> On Sat, May 27, 2017 at 8:57 PM Imran Rashid  wrote:
>
>> I had asked zoltan to bring this discussion to the dev list because I
>> think it's a question that extends beyond a single jira (we can't figure
>> out the semantics of timestamp in parquet if we don't k ow the overall goal
>> of the timestamp type) and since its a design question the entire community
>> should be involved.
>>
>> I think that a lot of the confusion comes because we're talking about
>> different ways time zone affect behavior: (1) parsing and (2) behavior when
>> changing time zones for processing data.
>>
>> It seems we agree that spark should eventually provide a timestamp type
>> which does conform to the standard.   The question is, how do we get
>> there?  Has spark already broken compliance so much that it's impossible to
>> go back without breaking user behavior?  Or perhaps spark already has
>> inconsistent behavior / broken compatibility within the 2.x line, so its
>> not unthinkable to have another breaking change?
>>
>> (Another part of the confusion is on me -- I believed the behavior change
>> was in 2.2, but actually it looks like its in 2.0.1.  That changes how we
>> think about this in context of what goes into a 2.2 release.  SPARK-18350
>> isn't the origin of the difference in behavior.)
>>
>> First: consider processing data that is already stored in tables, and
>> then accessing it from machines in different time zones.  The standard is
>> clear that "timestamp" should be just like "timestamp without time zone":
>> it does not represent one instant in time, rather it's always displayed the
>> same, regardless of time zone.  This was the behavior in spark 2.0.0 (and
>> 1.6),  for hive tables stored as text files, and for spark's json formats.
>>
>> Spark 2.0.1  changed the behavior of the json format (I believe
>> with SPARK-16216), so that it behaves more like timestamp *with* time
>> zone.  It also makes csv behave the same (timestamp in csv was basically
>> broken in 2.0.0).  However it did *not* change the behavior of a hive
>> textfile; it still behaves like "timestamp with*out* time zone".  Here's
>> some experiments I tried -- there are a bunch of files there for
>> completeness, but mostly focus on the difference between
>> query_output_2_0_0.txt vs. query_output_2_0_1.txt
>>
>> https://gist.github.com/squito/f348508ca7903ec2e1a64f4233e7aa70
>>
>> Given that spark has changed this behavior post 2.0.0, is it still out of
>> the question to change this behavior to bring it back in line with the sql
>> standard for timestamp (without time zone) in the 2.x line?  Or, as reynold
>> proposes, is the only option at this point to add an off-by-default feature
>> flag to get "timestamp without time zone" semantics?
>>
>>
>> Second, there is the question of parsing strings into timestamp type.
>> I'm far less knowledgeable about this, so I mostly just have questions:
>>
>> * does the standard dictate what the parsing behavior should be for
>> timestamp (without time zone) when a time zone is present?
>>
>> * if it does and spark violates this standard is it worth trying to
>> retain the *other* semantics of timestamp without time zone, even if we
>> violate the parsing part?
>>
>> I did look at what postgres does for comparison:
>>
>> https://gist.github.com/squito/cb81a1bb07e8f67e9d27eaef44cc522c
>>
>> spark's timestamp certainly does not 

Re: SQL TIMESTAMP semantics vs. SPARK-18350

2017-06-01 Thread Michael Allman
I would suggest that making timestamp type behavior configurable and persisted 
per-table could introduce some real confusion, e.g. in queries involving tables 
with different timestamp type semantics.

I suggest starting with the assumption that timestamp type behavior is a 
per-session flag that can be set in a global `spark-defaults.conf` and consider 
more granular levels of configuration as people identify solid use cases.

Cheers,

Michael


> On May 30, 2017, at 7:41 AM, Zoltan Ivanfi  wrote:
> 
> Hi,
> 
> If I remember correctly, the TIMESTAMP type had UTC-normalized local time 
> semantics even before Spark 2, so I can understand that Spark considers it to 
> be the "established" behavior that must not be broken. Unfortunately, this 
> behavior does not provide interoperability with other SQL engines of the 
> Hadoop stack.
> 
> Let me summarize the findings of this e-mail thread so far:
> Timezone-agnostic TIMESTAMP semantics would be beneficial for 
> interoperability and SQL compliance.
> Spark can not make a breaking change. For backward-compatibility with 
> existing data, timestamp semantics should be user-configurable on a per-table 
> level.
> Before going into the specifics of a possible solution, do we all agree on 
> these points?
> 
> Thanks,
> 
> Zoltan
> 
> On Sat, May 27, 2017 at 8:57 PM Imran Rashid  > wrote:
> I had asked zoltan to bring this discussion to the dev list because I think 
> it's a question that extends beyond a single jira (we can't figure out the 
> semantics of timestamp in parquet if we don't k ow the overall goal of the 
> timestamp type) and since its a design question the entire community should 
> be involved.
> 
> I think that a lot of the confusion comes because we're talking about 
> different ways time zone affect behavior: (1) parsing and (2) behavior when 
> changing time zones for processing data.
> 
> It seems we agree that spark should eventually provide a timestamp type which 
> does conform to the standard.   The question is, how do we get there?  Has 
> spark already broken compliance so much that it's impossible to go back 
> without breaking user behavior?  Or perhaps spark already has inconsistent 
> behavior / broken compatibility within the 2.x line, so its not unthinkable 
> to have another breaking change?
> 
> (Another part of the confusion is on me -- I believed the behavior change was 
> in 2.2, but actually it looks like its in 2.0.1.  That changes how we think 
> about this in context of what goes into a 2.2 release.  SPARK-18350 isn't the 
> origin of the difference in behavior.)
> 
> First: consider processing data that is already stored in tables, and then 
> accessing it from machines in different time zones.  The standard is clear 
> that "timestamp" should be just like "timestamp without time zone": it does 
> not represent one instant in time, rather it's always displayed the same, 
> regardless of time zone.  This was the behavior in spark 2.0.0 (and 1.6),  
> for hive tables stored as text files, and for spark's json formats.
> 
> Spark 2.0.1  changed the behavior of the json format (I believe with 
> SPARK-16216), so that it behaves more like timestamp *with* time zone.  It 
> also makes csv behave the same (timestamp in csv was basically broken in 
> 2.0.0).  However it did *not* change the behavior of a hive textfile; it 
> still behaves like "timestamp with*out* time zone".  Here's some experiments 
> I tried -- there are a bunch of files there for completeness, but mostly 
> focus on the difference between query_output_2_0_0.txt vs. 
> query_output_2_0_1.txt
> 
> https://gist.github.com/squito/f348508ca7903ec2e1a64f4233e7aa70 
> 
> 
> Given that spark has changed this behavior post 2.0.0, is it still out of the 
> question to change this behavior to bring it back in line with the sql 
> standard for timestamp (without time zone) in the 2.x line?  Or, as reynold 
> proposes, is the only option at this point to add an off-by-default feature 
> flag to get "timestamp without time zone" semantics?
> 
> 
> Second, there is the question of parsing strings into timestamp type.   I'm 
> far less knowledgeable about this, so I mostly just have questions:
> 
> * does the standard dictate what the parsing behavior should be for timestamp 
> (without time zone) when a time zone is present?
> 
> * if it does and spark violates this standard is it worth trying to retain 
> the *other* semantics of timestamp without time zone, even if we violate the 
> parsing part?
> 
> I did look at what postgres does for comparison:
> 
> https://gist.github.com/squito/cb81a1bb07e8f67e9d27eaef44cc522c 
> 
> 
> spark's timestamp certainly does not match postgres's timestamp for parsing, 
> it seems closer to postgres's "timestamp with timezone" -- though I dunno if 
> that is standard behavior at 

[Spark SQL] Nanoseconds in Timestamps are set as Microseconds

2017-06-01 Thread Anton Okolnychyi
Hi all,

I would like to ask what the community thinks regarding the way how Spark
handles nanoseconds in the Timestamp type.

As far as I see in the code, Spark assumes microseconds precision.
Therefore, I expect to have a truncated to microseconds timestamp or an
exception if I specify a timestamp with nanoseconds. However, the current
implementation just silently sets nanoseconds as microseconds in [1], which
results in a wrong timestamp. Consider the example below:

spark.sql("SELECT cast('2015-01-02 00:00:00.1' as
TIMESTAMP)").show(false)
++
|CAST(2015-01-02 00:00:00.1 AS TIMESTAMP)|
++
|2015-01-02 00:00:00.01  |
++

This issue was already raised in SPARK-17914 but I do not see any decision
there.

[1] - org.apache.spark.sql.catalyst.util.DateTimeUtils, toJavaTimestamp,
line 204

Best regards,
Anton