You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Pavel Pereslegin (Jira)" <ji...@apache.org> on 2023/04/26 08:08:00 UTC

[jira] [Comment Edited] (IGNITE-19274) Sql. Jdbc side working with TIMESTAMP WITH LOCAL TIME ZONE did not take into account current tz while storing data.

    [ https://issues.apache.org/jira/browse/IGNITE-19274?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17716595#comment-17716595 ] 

Pavel Pereslegin edited comment on IGNITE-19274 at 4/26/23 8:07 AM:
--------------------------------------------------------------------

The issue described in the task is one of the problems in terms of supporting work with the client's timezone.

Currently we have the following types:
||Calcite type||JDBC Type||Current Ignite type||Type contains timezone info||
|TIME|java.sql.Time|java.time.LocalTime|no|
|TIME_WITH_LOCAL_TIME_ZONE|java.sql.Time|java.time.LocalTime|no|
|TIMESTAMP|java.sql.Timestamp|java.time.LocalDateTime|no|
|TIMESTAMP_WITH_LOCAL_TIME_ZONE|java.sql.Timestamp|java.time.Instant|yea|

 

*{*}_WITH_LOCAL_TIME_ZONE types must take into account the client's local time zone.{*} Those. if the client is in the "GMT+3" time zone and is passing the value "14:00", the value must be converted from GMT+3 to UTC and the value "11:00" must be stored in the database.

Currently we do not take into account the client's/initiator's timezone in any way.
h2. Proposal.

There are two implementation options, it all depends on when we will transfer the client's time zone.
h2. 1. Transfer TimeZone from client per connection.

 In this case the timezone from client will be stored per connection.
{code:java}
 connect()
   // store client's time zone
   
 insert (time with local time zone '14:00')
   // store: value - timezone_offset = '11:00' (UTC)
   
 select time with local time zone
   // read: value from DB '11:00' UTC
   // transfer: value + timezone_offset
   // returns: LocalTime
   
 getTime()
   // return value{code}
In this case all offsets will be applied on the server side, those timestamp_with_time_zone we'll need to be LocalDateTime.
Since the timezone will only be set for each connection. Changing the client's time zone during connection will have no effect.
E,g,
{code:java}
set timeZone='GMT+1'
connect()
insert (time with local time zone '14:00')
select time -> returns '13:00'
set timeZone='GMT+2'
select time -> returns '13:00'
{code}
 
h2. 2. Transfer TimeZone from client per query.

In this case, we will need to add some special type instead of LocalTime because it does not store timezone information.
Since we will have time zone information on the client, we will simply translate the value to the local ghb time zone when it is displayed.
 
Those. the scheme of work is approximately the following.
{code:java}
 connect()
 
 insert (time with local time zone '14:00')
   // put client's timezone from the request to execution context.
   // store: value - timezone_offset = '11:00' (UTC)
 
 select time with local time zone
   // read: value from DB '11:00' UTC
   // returns: LocalTimeTz
   
 getTime()
   // if (value instanceof LocalTimeTz)
   //    read UTC '11:00' + local_timezone_offset{code}
 But usually no one does this, the time zone is transmitted when connected by the server.
 And we need to separately discover how it will work with a Ignite thin client.

 


was (Author: xtern):
The issue described in the task is one of the problems in terms of supporting work with the client's timezone.

Currently we have the following types:

 
||Calcite type||JDBC Type||Current Ignite type||Type contains timezone info||
|TIME|java.sql.Time|java.time.LocalTime|no|
|TIME_WITH_LOCAL_TIME_ZONE|java.sql.Time|java.time.LocalTime|no|
|TIMESTAMP|java.sql.Timestamp|java.time.LocalDateTime|no|
|TIMESTAMP_WITH_LOCAL_TIME_ZONE|java.sql.Timestamp|java.time.Instant|yea|

 

**_WITH_LOCAL_TIME_ZONE types must take into account the client's local time zone.* Those. if the client is in the "GMT+3" time zone and is passing the value "14:00", the value must be converted from GMT+3 to UTC and the value "11:00" must be stored in the database.

Currently we do not take into account the client's/initiator's timezone in any way.
h2. Proposal.

There are two implementation options, it all depends on when we will transfer the client's time zone.
h2. 1. Transfer TimeZone from client per connection.

 In this case the timezone from client will be stored per connection.
{code:java}
 connect()
   // store client's time zone
   
 insert (time with local time zone '14:00')
   // store: value - timezone_offset = '11:00' (UTC)
   
 select time with local time zone
   // read: value from DB '11:00' UTC
   // transfer: value + timezone_offset
   // returns: LocalTime
   
 getTime()
   // return value{code}
  
In this case all offsets will be applied on the server side, those timestamp_with_time_zone we'll need to be LocalDateTime.
Since the timezone will only be set for each connection. Changing the client's time zone during connection will have no effect.
E,g,

 

 
{code:java}
set timeZone='GMT+1'
connect()
insert (time with local time zone '14:00')
select time -> returns '13:00'
set timeZone='GMT+2'
select time -> returns '13:00'
{code}
 
h2. 2. Transfer TimeZone from client per query.

In this case, we will need to add some special type instead of LocalTime because it does not store timezone information.
Since we will have time zone information on the client, we will simply translate the value to the local ghb time zone when it is displayed.
 
Those. the scheme of work is approximately the following.
{code:java}
 connect()
 
 insert (time with local time zone '14:00')
   // put client's timezone from the request to execution context.
   // store: value - timezone_offset = '11:00' (UTC)
 
 select time with local time zone
   // read: value from DB '11:00' UTC
   // returns: LocalTimeTz
   
 getTime()
   // if (value instanceof LocalTimeTz)
   //    read UTC '11:00' + local_timezone_offset{code}
 But usually no one does this, the time zone is transmitted when connected by the server.
 And we need to separately discover how it will work with a Ignite thin client.

 

> Sql. Jdbc side working with TIMESTAMP WITH LOCAL TIME ZONE did not take into account current tz while storing data.
> -------------------------------------------------------------------------------------------------------------------
>
>                 Key: IGNITE-19274
>                 URL: https://issues.apache.org/jira/browse/IGNITE-19274
>             Project: Ignite
>          Issue Type: Bug
>          Components: sql
>    Affects Versions: 3.0.0-beta1
>            Reporter: Evgeny Stanilovsky
>            Assignee: Pavel Pereslegin
>            Priority: Major
>
> The {{TIMESTAMP WITH LOCAL TIME ZONE}} data type is a variant of {{TIMESTAMP}} that includes a time zone offset in its value. Data stored in the database is normalized to the database time zone (UTC) and time zone offset is not stored as part of the column data. When the data is retrieved, it to be returned in the user's local session time zone.
> i.e:
> {noformat}
> CREATE TABLE timestamp(ts TIMESTAMP, t_tz TIMESTAMP WITH TIME ZONE);
> SET TIME ZONE 'tz1';
> INSERT INTO timestamp VALUES ('2011-01-01 01:01:01', TIMESTAMP WITH TIME ZONE '2011-01-01 01:01:01');
> SET TIME ZONE 'tz2';
> INSERT INTO timestamp VALUES ('2011-01-01 01:01:01', TIMESTAMP WITH TIME ZONE '2011-01-01 01:01:01');
> ...
> select * from timestamp;{noformat}
> returned rows need to be different in case of different tz1 and tz2 offsets but they are equals for now. Also returned representation need to be present in user session time zone.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)