Skip to content

Commit

Permalink
[FLINK-22428][docs][table] Translate "Timezone" page into Chinese (#1…
Browse files Browse the repository at this point in the history
  • Loading branch information
hackergin authored and wuchong committed Apr 28, 2021
1 parent 928871b commit 4885c4c
Show file tree
Hide file tree
Showing 2 changed files with 67 additions and 69 deletions.
128 changes: 63 additions & 65 deletions docs/content.zh/docs/dev/table/concepts/timezone.md
Original file line number Diff line number Diff line change
Expand Up @@ -23,29 +23,29 @@ specific language governing permissions and limitations
under the License.
-->

## Overview
## 概述

Flink provides rich data types for Date and Time, including `DATE`, `TIME`, `TIMESTAMP`, `TIMESTAMP_LTZ`, `INTERVAL YEAR TO MONTH`, `INTERVAL DAY TO SECOND` (please see [Date and Time]({{< ref "docs/dev/table/types" >}}#date-and-time) for detailed information).
Flink supports setting time zone in session level (please see [table.local-time-zone]({{< ref "docs/dev/table/config">}}#table-local-time-zone) for detailed information).
These timestamp data types and time zone support of Flink make it easy to process business data across time zones.
Flink 为日期和时间提供了丰富的数据类型, 包括 `DATE` `TIME` `TIMESTAMP` `TIMESTAMP_LTZ` `INTERVAL YEAR TO MONTH` `INTERVAL DAY TO SECOND` (更多详情请参考 [Date and Time]({{< ref "docs/dev/table/types" >}}#date-and-time))。
Flink 支持在 session (会话)级别设置时区(更多详情请参考 [table.local-time-zone]({{< ref "docs/dev/table/config">}}#table-local-time-zone))。
Flink 对多种时间类型和时区的支持使得跨时区的数据处理变得非常容易。

## TIMESTAMP vs TIMESTAMP_LTZ

### TIMESTAMP type
- `TIMESTAMP(p)` is an abbreviation for `TIMESTAMP(p) WITHOUT TIME ZONE`, the precision `p` supports range is from 0 to 9, 6 by default.
- `TIMESTAMP` describes a timestamp represents year, month, day, hour, minute, second and fractional seconds.
- `TIMESTAMP` can be specified from a string literal, e.g.
### TIMESTAMP 类型
- `TIMESTAMP(p)` `TIMESTAMP(p) WITHOUT TIME ZONE` 的简写, 精度 `p` 支持的范围是0-9, 默认是6。
- `TIMESTAMP` 用于描述年, 月, 日, 小时, 分钟, 秒 和 小数秒对应的时间戳。
- `TIMESTAMP` 可以通过一个字符串来指定,例如:
```sql
Flink SQL> SELECT TIMESTAMP '1970-01-01 00:00:04.001';
+-------------------------+
| 1970-01-01 00:00:04.001 |
+-------------------------+
```

### TIMESTAMP_LTZ type
- `TIMESTAMP_LTZ(p)` is an abbreviation for `TIMESTAMP(p) WITH LOCAL TIME ZONE`, the precision `p` supports range is from 0 to 9, 6 by default.
- `TIMESTAMP_LTZ` describes an absolute time point on the time-line, it stores a long value representing epoch-milliseconds and an int representing nanosecond-of-millisecond. The epoch time is measured from the standard Java epoch of `1970-01-01T00:00:00Z`. Every datum of `TIMESTAMP_LTZ` type is interpreted in the local time zone configured in the current session for computation and visualization.
- `TIMESTAMP_LTZ` has no literal representation and thus can not specify from literal, it can derives from a long epoch time(e.g. The long time produced by Java `System.currentTimeMillis()`)
### TIMESTAMP_LTZ 类型
- `TIMESTAMP_LTZ(p)` `TIMESTAMP(p) WITH LOCAL TIME ZONE` 的简写, 精度 `p` 支持的范围是0-9, 默认是6。
- `TIMESTAMP_LTZ` 用于描述时间线上的绝对时间点, 使用 long 保存从 epoch 至今的毫秒数, 使用int保存毫秒中的纳秒数。 epoch 时间是从 java 的标准 epoch 时间 `1970-01-01T00:00:00Z` 开始计算。 在计算和可视化时, 每个 `TIMESTAMP_LTZ` 类型的数据都是使用的 session (会话)中配置的时区。
- `TIMESTAMP_LTZ` 没有字符串表达形式因此无法通过字符串来指定, 可以通过一个 long 类型的 epoch 时间来转化(例如: 通过 Java 来产生一个 long 类型的 epoch 时间 `System.currentTimeMillis()`)

```sql
Flink SQL> CREATE VIEW T1 AS SELECT TO_TIMESTAMP_LTZ(4001, 3);
Expand All @@ -66,22 +66,22 @@ Flink SQL> SELECT * FROM T1;
+---------------------------+
```

- `TIMESTAMP_LTZ` can be used in cross time zones business because the absolute time point (e.g. above `4001` milliseconds) describes a same instantaneous point in different time zones.
Giving a background that at a same time point, the `System.currentTimeMillis()` of all machines in the world returns same value (e.g. the `4001` milliseconds in above example), this is absolute time point meaning.
- `TIMESTAMP_LTZ` 可以用于跨时区的计算,因为它是一个基于 epoch 的绝对时间点(比如上例中的 `4001` 毫秒)代表的就是不同时区的同一个绝对时间点。
补充一个背景知识:在同一个时间点, 全世界所有的机器上执行 `System.currentTimeMillis()` 都会返回同样的值。 (比如上例中的 `4001` milliseconds), 这就是绝对时间的定义。

## Time Zone Usage
The local time zone defines current session time zone id. You can config the time zone in Sql Client or Applications.
## 时区的作用
本地时区定义了当前 session(会话)所在的时区, 你可以在 Sql client 或者应用程序中配置。

{{< tabs "SQL snippets" >}}
{{< tab "SQL Client" >}}
```sql
# set to UTC time zone
-- 设置为 UTC 时区
Flink SQL> SET table.local-time-zone=UTC;

# set to Shanghai time zone
-- 设置为上海时区
Flink SQL> SET table.local-time-zone=Asia/Shanghai;

# set to Los_Angeles time zone
-- 设置为Los_Angeles时区
Flink SQL> SET table.local-time-zone=America/Los_Angeles;
```
{{< /tab >}}
Expand All @@ -90,13 +90,13 @@ Flink SQL> SET table.local-time-zone=America/Los_Angeles;
EnvironmentSettings envSetting = EnvironmentSettings.newInstance().build();
TableEnvironment tEnv = TableEnvironment.create(envSetting);

// set to UTC time zone
// 设置为 UTC 时区
tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC"));

// set to Shanghai time zone
// 设置为上海时区
tEnv.getConfig().setLocalTimeZone(ZoneId.of("Asia/Shanghai"));

// set to Los_Angeles time zone
// 设置为 Los_Angeles 时区
tEnv.getConfig().setLocalTimeZone(ZoneId.of("America/Los_Angeles"));
```
{{< /tab >}}
Expand All @@ -105,22 +105,22 @@ Flink SQL> SET table.local-time-zone=America/Los_Angeles;
val envSetting = EnvironmentSettings.newInstance.build
val tEnv = TableEnvironment.create(envSetting)

// set to UTC time zone
// 设置为 UTC 时区
tEnv.getConfig.setLocalTimeZone(ZoneId.of("UTC"))

// set to Shanghai time zone
// 设置为上海时区
tEnv.getConfig.setLocalTimeZone(ZoneId.of("Asia/Shanghai"))

// set to Los_Angeles time zone
// 设置为 Los_Angeles 时区
tEnv.getConfig.setLocalTimeZone(ZoneId.of("America/Los_Angeles"))
```
{{< /tab >}}
{{< /tabs >}}

The session time zone is useful in Flink SQL, the main usages are:
session(会话)的时区设置在 Flink SQL 中非常有用, 它的主要用法如下:

### Decide time functions return value
The following time functions is influenced by the configured time zone.
### 确定时间函数的返回值
session (会话)中配置的时区会对以下函数生效。
* LOCALTIME
* LOCALTIMESTAMP
* CURRENT_DATE
Expand Down Expand Up @@ -178,8 +178,8 @@ Flink SQL> SELECT * FROM MyView1;
+-----------+-------------------------+--------------+--------------+-------------------------+-------------------------+-------------------------+-------------------------+
```

### `TIMESTAMP_LTZ` string representation
The session timezone is used when represents a `TIMESTAMP_LTZ` value to string format, i.e print the value, cast the value to `STRING` type, cast the value to `TIMESTAMP`, cast a `TIMESTAMP` value to `TIMESTAMP_LTZ`:
### `TIMESTAMP_LTZ` 字符串表示
当一个 `TIMESTAMP_LTZ` 值转为 string 格式时, session 中配置的时区会生效。 例如打印这个值,将类型强制转化为 `STRING` 类型, 将类型强制转换为 `TIMESTAMP` ,将 `TIMESTAMP` 的值转化为 `TIMESTAMP_LTZ` 类型:
```sql
Flink SQL> CREATE VIEW MyView2 AS SELECT TO_TIMESTAMP_LTZ(4001, 3) AS ltz, TIMESTAMP '1970-01-01 00:00:01.001' AS ntz;
Flink SQL> DESC MyView2;
Expand Down Expand Up @@ -249,20 +249,19 @@ Flink SQL> SELECT * FROM MyView3;
+-------------------------+---------------------------+-------------------------+-------------------------+-------------------------------+
```

## Time Attribute and Time Zone
## 时间属性和时区
更多时间属性相关的详细介绍, 请参考 [Time Attribute]({{< ref "docs/dev/table/concepts/time_attributes">}}#时间属性) 。

Please see [Time Attribute]({{< ref "docs/dev/table/concepts/time_attributes">}}#时间属性) for more information about time attribute.

### Processing Time and Time Zone
Flink SQL defines process time attribute by function `PROCTIME()`, the function return type is `TIMESTAMP_LTZ`.
### 处理时间和时区
Flink SQL 使用函数 `PROCTIME()` 来定义处理时间属性, 该函数返回的类型是 `TIMESTAMP_LTZ`

{{< hint info >}}
Before Flink 1.13, the function return type of `PROCTIME()` is `TIMESTAMP`, and the return value is the `TIMESTAMP` in UTC time zone,
e.g. the wall-clock shows `2021-03-01 12:00:00` at Shanghai, however the `PROCTIME()` displays `2021-03-01 04:00:00` which is wrong.
Flin 1.13 fixes this issue and uses `TIMESTAMP_LTZ` type as return type of `PROCTIME()`, users don't need to deal time zone problems anymore.
在 Flink1.13 之前, `PROCTIME()` 函数返回的类型是 `TIMESTAMP` , 返回值是UTC时区下的 `TIMESTAMP`
例如: 当上海的时间为 `2021-03-01 12:00:00` 时, `PROCTIME()` 显示的时间却是错误的 `2021-03-01 04:00:00`
这个问题在 Flink 1.13 中修复了, 因此用户不用再去处理时区的问题了。
{{< /hint >}}

The PROCTIME() always represents your local timestamp value, using TIMESTAMP_LTZ type can also support DayLight Saving Time well.
`PROCTIME()` 返回的是本地时区的时间, 使用 `TIMESTAMP_LTZ` 类型也可以支持夏令时时间。

```sql
Flink SQL> SET table.local-time-zone=UTC;
Expand Down Expand Up @@ -325,7 +324,7 @@ Flink SQL> DESC MyView3;
+-----------------+-----------------------------+-------+-----+--------+-----------+
```

Use the following command to ingest data for `MyTable1` in a terminal:
在终端执行以下命令写入数据到 `MyTable1`

```
> nc -lk 9999
Expand Down Expand Up @@ -356,7 +355,7 @@ Flink SQL> SET table.local-time-zone=Asia/Shanghai;
Flink SQL> SELECT * FROM MyView3;
```

Returns the different window start, window end and window proctime compared to calculation in UTC timezone.
相比在 UTC 时区下的计算结果, 在 Asia/Shanghai 时区下计算的窗口开始时间, 窗口结束时间和窗口处理时间是不同的。
```
+-------------------------+-------------------------+-------------------------+------+-----------+
| window_start | window_end | window_procime | item | max_price |
Expand All @@ -368,14 +367,14 @@ Returns the different window start, window end and window proctime compared to c
```

{{< hint info >}}
Processing time window is non-deterministic, so each run will get different windows and different aggregations. The above example is just for explaining how time zone affects processing time window.
处理时间窗口是不确定的, 每次运行都会返回不同的窗口和聚合结果。 以上的示例只用于说明时区如何影响处理时间窗口。
{{< /hint >}}

### Event Time and Time Zone
Flink supports defining event time attribute on TIMESTAMP column and TIMESTAMP_LTZ column.
### 事件时间和时区
Flink 支持在 `TIMESTAMP` 列和 `TIMESTAMP_LTZ` 列上定义时间属性。

#### Event Time Attribute on TIMESTAMP
If the timestamp data in the source is represented as year-month-day-hour-minute-second, usually a string value without time-zone information, e.g. `2020-04-15 20:13:40.564`, it's recommended to define the event time attribute as a `TIMESTAMP` column:
#### TIMESTAMP 上的事件时间属性
如果 source 中的时间用于表示年-月-日-小时-分钟-秒, 通常是一个不带时区的字符串, 例如: `2020-04-15 20:13:40.564`。 推荐在 `TIMESTAMP` 列上定义事件时间属性。
```sql
Flink SQL> CREATE TABLE MyTable2 (
item STRING,
Expand Down Expand Up @@ -414,7 +413,7 @@ Flink SQL> DESC MyView4;
+----------------+------------------------+------+-----+--------+-----------+
```

Use the following command to ingest data for `MyTable2` in a terminal:
在终端执行以下命令用于写入数据到 `MyTable2`

```
> nc -lk 9999
Expand Down Expand Up @@ -446,7 +445,7 @@ Flink SQL> SET table.local-time-zone=Asia/Shanghai;
Flink SQL> SELECT * FROM MyView4;
```

Returns the same window start, window end and window rowtime compared to calculation in UTC timezone.
相比在 UTC 时区下的计算结果, 在 Asia/Shanghai 时区下计算的窗口开始时间, 窗口结束时间和窗口的 rowtime 是相同的。
```
+-------------------------+-------------------------+-------------------------+------+-----------+
| window_start | window_end | window_rowtime | item | max_price |
Expand All @@ -457,8 +456,8 @@ Returns the same window start, window end and window rowtime compared to calcula
+-------------------------+-------------------------+-------------------------+------+-----------+
```

#### Event Time Attribute on TIMESTAMP_LTZ
If the timestamp data in the source is represented as a epoch time, usually a long value, e.g. `1618989564564`, it's recommended to define event time attribute as a `TIMESTAMP_LTZ` column.
#### TIMESTAMP_LTZ 上的事件时间属性
如果源数据中的时间为一个 epoch 时间, 通常是一个 long 值, 例如: `1618989564564` ,推荐将事件时间属性定义在 `TIMESTAMP_LTZ` 列上。
```sql
Flink SQL> CREATE TABLE MyTable3 (
item STRING,
Expand Down Expand Up @@ -498,7 +497,7 @@ Flink SQL> DESC MyView5;
+----------------+----------------------------+-------+-----+--------+-----------+
```

The input data of MyTable3 is:
`MyTable3` 的输入数据为:
```
A,1.1,1618495260000 # The corresponding utc timestamp is 2021-04-15 14:01:00
B,1.2,1618495320000 # The corresponding utc timestamp is 2021-04-15 14:02:00
Expand Down Expand Up @@ -528,7 +527,7 @@ Flink SQL> SET table.local-time-zone=Asia/Shanghai;
Flink SQL> SELECT * FROM MyView5;
```

Returns the different window start, window end and window rowtime compared to calculation in UTC timezone.
相比在 UTC 时区下的计算结果, 在 Asia/Shanghai 时区下计算的窗口开始时间, 窗口结束时间和窗口的 rowtime 是不同的。
```
+-------------------------+-------------------------+-------------------------+------+-----------+
| window_start | window_end | window_rowtime | item | max_price |
Expand All @@ -539,35 +538,34 @@ Returns the different window start, window end and window rowtime compared to ca
+-------------------------+-------------------------+-------------------------+------+-----------+
```

## Daylight Saving Time Support
Flink SQL supports defining time attributes on TIMESTAMP_LTZ column, base on this, Flink SQL gracefully uses TIMESTAMP and TIMESTAMP_LTZ type in window processing to support the Daylight Saving Time.


Flink use timestamp literal to split the window and assigns window to data according to the epoch time of the each row. It means Flink uses `TIMESTAMP` type for window start and window end (e.g. `TUMBLE_START` and `TUMBLE_END`), uses `TIMESTAMP_LTZ` for window time attribute (e.g. `TUMBLE_PROCTIME`, `TUMBLE_ROWTIME`).
Given a example of tumble window, the DaylightTime in Los_Angele starts at time 2021-03-14 02:00:00:
## 夏令时支持
Flink SQL支持在 `TIMESTAMP_LTZ`列上定义时间属性, 基于这一特征,Flink SQL 在窗口中使用 `TIMESTAMP``TIMESTAMP_LTZ` 类型优雅地支持了夏令时。

Flink 使用时间戳的字符格式来分割窗口并通过每条记录对应的 epoch 时间来分配窗口。 这意味着 Flink 窗口开始时间和窗口结束时间使用的是 `TIMESTAMP` 类型(例如: `TUMBLE_START``TUMBLE_END`), 窗口的时间属性使用的是 `TIMESTAMP_LTZ` 类型(例如: `TUMBLE_PROCTIME``TUMBLE_ROWTIME`)。
给定一个 tumble window示例, 在 Los_Angele 时区下夏令时从 `2021-03-14 02:00:00` 开始:
```
long epoch1 = 1615708800000L; // 2021-03-14 00:00:00
long epoch2 = 1615712400000L; // 2021-03-14 01:00:00
long epoch3 = 1615716000000L; // 2021-03-14 03:00:00, skip one hour (2021-03-14 02:00:00)
long epoch3 = 1615716000000L; // 2021-03-14 03:00:00, 手表往前拨一小时,跳过 (2021-03-14 02:00:00)
long epoch4 = 1615719600000L; // 2021-03-14 04:00:00
```
The tumble window [2021-03-14 00:00:00, 2021-03-14 00:04:00] will collect 3 hours' data in Los_angele time zone, but it collect 4 hours' data in other non-DST time zones, what user to do is only define time attribute on TIMESTAMP_LTZ column.
在 Los_angele 时区下, tumble window [2021-03-14 00:00:00, 2021-03-14 00:04:00] 将会收集3个小时的数据, 在其他非夏令时的时区下将会收集4个小时的数据,用户只需要在 `TIMESTAMP_LTZ` 列上声明时间属性即可。

All windows in Flink like Hop window, Session window, Cumulative window follow this way, and all operations in Flink SQL support TIMESTAMP_LTZ well, thus Flink gracefully supports the Daylight Saving Time zone.  
Flink 的所有窗口(如 Hop window Session window Cumulative window)都会遵循这种方式, Flink SQL 中的所有操作都很好地支持了 `TIMESTAMP_LTZ` 类型,因此Flink可以非常优雅的支持夏令时。  


## Difference between Batch and Streaming Mode
The following time functions:
## Batch 模式和 Streaming 模式的区别
以下函数:
* LOCALTIME
* LOCALTIMESTAMP
* CURRENT_DATE
* CURRENT_TIME
* CURRENT_TIMESTAMP
* NOW()

Flink evaluates their values according to execution mode. They are evaluated for each record in streaming mode. But in batch mode, they are evaluated once as the query starts and uses the same result for every row.
Flink 会根据执行模式来进行不同计算,在 Streaming 模式下这些函数是每条记录都会计算一次,但在 Batch 模式下,只会在 query 开始时计算一次,所有记录都使用相同的结果。

The following time functions are evaluated for each record no matter in batch or streaming mode:
以下时间函数无论是在 Streaming 模式还是 Batch 模式下,都会为每条记录计算一次结果:

* CURRENT_ROW_TIMESTAMP()
* PROCTIME()
Expand Down
8 changes: 4 additions & 4 deletions docs/content/docs/dev/table/concepts/timezone.md
Original file line number Diff line number Diff line change
Expand Up @@ -75,13 +75,13 @@ The local time zone defines current session time zone id. You can config the tim
{{< tabs "SQL snippets" >}}
{{< tab "SQL Client" >}}
```sql
# set to UTC time zone
-- set to UTC time zone
Flink SQL> SET table.local-time-zone=UTC;

# set to Shanghai time zone
-- set to Shanghai time zone
Flink SQL> SET table.local-time-zone=Asia/Shanghai;

# set to Los_Angeles time zone
-- set to Los_Angeles time zone
Flink SQL> SET table.local-time-zone=America/Los_Angeles;
```
{{< /tab >}}
Expand Down Expand Up @@ -259,7 +259,7 @@ Flink SQL defines process time attribute by function `PROCTIME()`, the function
{{< hint info >}}
Before Flink 1.13, the function return type of `PROCTIME()` is `TIMESTAMP`, and the return value is the `TIMESTAMP` in UTC time zone,
e.g. the wall-clock shows `2021-03-01 12:00:00` at Shanghai, however the `PROCTIME()` displays `2021-03-01 04:00:00` which is wrong.
Flin 1.13 fixes this issue and uses `TIMESTAMP_LTZ` type as return type of `PROCTIME()`, users don't need to deal time zone problems anymore.
Flink 1.13 fixes this issue and uses `TIMESTAMP_LTZ` type as return type of `PROCTIME()`, users don't need to deal time zone problems anymore.
{{< /hint >}}

The PROCTIME() always represents your local timestamp value, using TIMESTAMP_LTZ type can also support DayLight Saving Time well.
Expand Down

0 comments on commit 4885c4c

Please sign in to comment.