This is an automated email from the ASF dual-hosted git repository.

sjwiesman pushed a commit to branch release-1.14
in repository https://gitbox.apache.org/repos/asf/flink.git

commit fc8e269e9415a75fe9d1c5634eef2caaf059e1bc
Author: Leonard Xu <xbjt...@gmail.com>
AuthorDate: Sun Sep 26 16:28:25 2021 +0800

    [FLINK-23313][docs] Reintroduce temporal table function documentation
    
    This closes #17356
---
 .../{legacy.md => temporal_table_function.md}      |  19 ++--
 .../docs/dev/table/concepts/versioned_tables.md    | 108 ---------------------
 .../content.zh/docs/dev/table/sql/queries/joins.md |  43 ++++++--
 .../{legacy.md => temporal_table_function.md}      |  21 ++--
 docs/content/docs/dev/table/sql/queries/joins.md   |  43 ++++++--
 docs/content/docs/dev/table/tableApi.md            |   2 +-
 6 files changed, 84 insertions(+), 152 deletions(-)

diff --git a/docs/content.zh/docs/dev/table/concepts/legacy.md 
b/docs/content.zh/docs/dev/table/concepts/temporal_table_function.md
similarity index 84%
rename from docs/content.zh/docs/dev/table/concepts/legacy.md
rename to docs/content.zh/docs/dev/table/concepts/temporal_table_function.md
index 93e778d..05e577f 100644
--- a/docs/content.zh/docs/dev/table/concepts/legacy.md
+++ b/docs/content.zh/docs/dev/table/concepts/temporal_table_function.md
@@ -1,5 +1,5 @@
 ---
-title: "Legacy Features"
+title: "Temporal Table Function"
 weight: 1002
 type: docs
 aliases:
@@ -24,22 +24,17 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-# Legacy Features
-
-As Flink SQL has matured there are some features that have been replaced with 
more modern and better functioning substitutes.
-These legacy features remain documented here for those users that have not yet 
or are unable to, upgrade to the more modern variant.
-
 # Temporal Table Function
 
-The temporal table function is the legacy way of defining something akin to a 
[versioned table]({{< ref "docs/dev/table/concepts/versioned_tables" >}})
-that can be used in a temporal table join.
-Please define temporal joins using [versioned tables]({{< ref 
"docs/dev/table/concepts/versioned_tables" >}}) in new queries.
+A Temporal table function provides access to the version of a temporal table 
at a specific point in time.
+In order to access the data in a temporal table, one must pass a [time 
attribute]({{<ref "docs/dev/table/concepts/time_attributes">}}) that determines 
the version of the table that will be returned.
+Flink uses the SQL syntax of [table functions]({{<ref 
"docs/dev/table/functions/udfs" >}}#table-functions) to provide a way to 
express it.
 
 Unlike a versioned table, temporal table functions can only be defined on top 
of append-only streams 
 &mdash; it does not support changelog inputs.
 Additionally, a temporal table function cannot be defined in pure SQL DDL. 
-
-#### Defining a Temporal Table Function
+ 
+## Defining a Temporal Table Function
 
 Temporal table functions can be defined on top of append-only streams using 
the [Table API]({{< ref "docs/dev/table/tableApi" >}}).
 The table is registered with one or more key columns, and a time attribute 
used for versioning.
@@ -83,7 +78,7 @@ tEnv.registerFunction("rates", rates)
 {{< /tab >}}
 {{< /tabs >}}
 
-#### Temporal Table Function Join
+## Temporal Table Function Join
 
 Once defined, a temporal table function is used as a standard [table 
function]({{< ref "docs/dev/table/functions/udfs" >}}#table-functions).
 Append-only tables (left input/probe side) can join with a temporal table 
(right input/build side),
diff --git a/docs/content.zh/docs/dev/table/concepts/versioned_tables.md 
b/docs/content.zh/docs/dev/table/concepts/versioned_tables.md
index 3823b32..f4acf0c 100644
--- a/docs/content.zh/docs/dev/table/concepts/versioned_tables.md
+++ b/docs/content.zh/docs/dev/table/concepts/versioned_tables.md
@@ -254,112 +254,4 @@ currency_time currency   rate
 
 在基于处理时间的时态表 Join 中支持任意表作为时态表会在不远的将来支持。
 
-时态表函数
-------------------------
-时态表函数是一种过时的方式去定义时态表并关联时态表的数据,现在我们可以用时态表 DDL 去定义时态表,用[时态表 Join]({{< ref 
"docs/dev/table/sql/queries/joins" >}}#时态表-join) 语法去关联时态表。
-
-时态表函数和时态表 DDL 最大的区别在于,时态表 DDL 可以在纯 SQL 环境中使用但是时态表函数不支持,用时态表 DDL 声明的时态表支持 
changelog 流和 append-only 流但时态表函数仅支持 append-only 流。
- 
-为了访问时态表中的数据,必须传递一个[时间属性]({{< ref "docs/dev/table/concepts/time_attributes" 
>}}),该属性确定将要返回的表的版本。
-Flink 使用[表函数]({{< ref "docs/dev/table/functions/udfs" >}}#表值函数)的 SQL 
语法提供一种表达它的方法。
-
-定义后,*时态表函数*将使用单个时间参数 timeAttribute 并返回一个行集合。
-该集合包含相对于给定时间属性的所有现有主键的行的最新版本。
-
-假设我们基于 `RatesHistory` 表定义了一个时态表函数,我们可以通过以下方式查询该函数 `Rates(timeAttribute)`: 
-
-```sql
-SELECT * FROM Rates('10:15:00');
-
-rowtime  currency  rate
-=======  ========= ====
-09:00:00 US Dollar 102
-09:00:00 Euro      114
-09:00:00 Yen       1
-
-SELECT * FROM Rates('11:00:00');
-
-rowtime  currency  rate
-======== ========= ====
-09:00:00 US Dollar 102
-10:45:00 Euro      116
-09:00:00 Yen       1
-```
-
-对 `Rates(timeAttribute)` 的每个查询都将返回给定 `timeAttribute` 的 `Rates` 状态。
-
-**注意**:当前 Flink 不支持使用常量时间属性参数直接查询时态表函数。目前,时态表函数只能在 join 中使用。上面的示例用于为函数 
`Rates(timeAttribute)` 返回内容提供直观信息。
-
-另请参阅有关[用于持续查询的 join ]({{< ref "docs/dev/table/sql/queries/joins" 
>}})页面,以获取有关如何与时态表 join 的更多信息。
-
-### 定义时态表函数
-
-以下代码段说明了如何从 append-only 表中创建时态表函数。
-
-{{< tabs "53d51b01-eee7-49b7-965d-98ab237fb3a1" >}}
-{{< tab "Java" >}}
-```java
-import org.apache.flink.table.functions.TemporalTableFunction;
-(...)
-
-// 获取 stream 和 table 环境
-StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
-
-// 提供一个汇率历史记录表静态数据集
-List<Tuple2<String, Long>> ratesHistoryData = new ArrayList<>();
-ratesHistoryData.add(Tuple2.of("US Dollar", 102L));
-ratesHistoryData.add(Tuple2.of("Euro", 114L));
-ratesHistoryData.add(Tuple2.of("Yen", 1L));
-ratesHistoryData.add(Tuple2.of("Euro", 116L));
-ratesHistoryData.add(Tuple2.of("Euro", 119L));
-
-// 用上面的数据集创建并注册一个示例表
-// 在实际设置中,应使用自己的表替换它
-DataStream<Tuple2<String, Long>> ratesHistoryStream = 
env.fromCollection(ratesHistoryData);
-Table ratesHistory = tEnv.fromDataStream(ratesHistoryStream, $("r_currency"), 
$("r_rate"), $("r_proctime").proctime());
-
-tEnv.createTemporaryView("RatesHistory", ratesHistory);
-
-// 创建和注册时态表函数
-// 指定 "r_proctime" 为时间属性,指定 "r_currency" 为主键
-TemporalTableFunction rates = 
ratesHistory.createTemporalTableFunction("r_proctime", "r_currency"); // <==== 
(1)
-tEnv.registerFunction("Rates", rates);                                         
                     // <==== (2)
-```
-{{< /tab >}}
-{{< tab "Scala" >}}
-```scala
-// 获取 stream 和 table 环境
-val env = StreamExecutionEnvironment.getExecutionEnvironment
-val tEnv = StreamTableEnvironment.create(env)
-
-// 提供一个汇率历史记录表静态数据集
-val ratesHistoryData = new mutable.MutableList[(String, Long)]
-ratesHistoryData.+=(("US Dollar", 102L))
-ratesHistoryData.+=(("Euro", 114L))
-ratesHistoryData.+=(("Yen", 1L))
-ratesHistoryData.+=(("Euro", 116L))
-ratesHistoryData.+=(("Euro", 119L))
-
-// 用上面的数据集创建并注册一个示例表
-// 在实际设置中,应使用自己的表替换它
-val ratesHistory = env
-  .fromCollection(ratesHistoryData)
-  .toTable(tEnv, 'r_currency, 'r_rate, 'r_proctime.proctime)
-
-tEnv.createTemporaryView("RatesHistory", ratesHistory)
-
-// 创建和注册时态表函数
-// 指定 "r_proctime" 为时间属性,指定 "r_currency" 为主键
-val rates = ratesHistory.createTemporalTableFunction($"r_proctime", 
$"r_currency") // <==== (1)
-tEnv.registerFunction("Rates", rates)                                          
// <==== (2)
-```
-{{< /tab >}}
-{{< /tabs >}}
-
-行`(1)`创建了一个 `rates` [时态表函数](#时态表函数),
-这使我们可以在[ Table API ]({{< ref "docs/dev/table/tableApi" >}}#joins)中使用 `rates` 
函数。
-
-行`(2)`在表环境中注册名称为 `Rates` 的函数,这使我们可以在[ SQL ]({{< ref 
"docs/dev/table/sql/queries/joins" >}})中使用 `Rates` 函数。
-
 {{< top >}}
diff --git a/docs/content.zh/docs/dev/table/sql/queries/joins.md 
b/docs/content.zh/docs/dev/table/sql/queries/joins.md
index 6e8e886..e14fdc6 100644
--- a/docs/content.zh/docs/dev/table/sql/queries/joins.md
+++ b/docs/content.zh/docs/dev/table/sql/queries/joins.md
@@ -109,9 +109,12 @@ Since time attributes are quasi-monotonic increasing, 
Flink can remove old value
 Temporal Joins
 --------------
 
+A Temporal table is a table that evolves over time - otherwise known in Flink 
as a [dynamic table]({% link dev/table/streaming/dynamic_tables.md %}). Rows in 
a temporal table are associated with one or more temporal periods and all Flink 
tables are temporal(dynamic).
+The temporal table contains one or more versioned table snapshots, it can be a 
changing history table which tracks the changes(e.g. database changelog, 
contains all snapshots) or a changing dimensioned table which materializes the 
changes(e.g. database table which contains the latest snapshot). 
+
 ### Event Time Temporal Join
 
-Temporal joins allow joining against a [versioned table]({{< ref 
"docs/dev/table/concepts/versioned_tables" >}}).
+Event Time Temporal joins allow joining against a [versioned table]({{< ref 
"docs/dev/table/concepts/versioned_tables" >}}).
 This means a table can be enriched with changing metadata and retrieve its 
value at a certain point in time. 
 
 Temporal joins take an arbitrary table (left input/probe site) and correlate 
each row to the corresponding row's relevant version in the versioned table 
(right input/build side). 
@@ -247,20 +250,19 @@ amount currency     rate   amount*rate
      2 Euro          116          232    <== arrived at time 10:52
 ```
 
-
-With the help of temporal table join, we can express such a query in SQL as:
+Currently, the `FOR SYSTEM_TIME AS OF` syntax used in temporal join with 
latest version of any view/table is not support yet, you can use temporal table 
function syntax as following:
 
 ```sql
 SELECT
-  o.amount, o.currency, r.rate, o.amount * r.rate
+  o_amount, r_rate
 FROM
-  Orders AS o
-  JOIN LatestRates FOR SYSTEM_TIME AS OF o.proctime AS r
-  ON r.currency = o.currency
+  Orders,
+  LATERAL TABLE (Rates(o_proctime))
+WHERE
+  r_currency = o_currency
 ```
 
-Each record from the probe side will be joined with the current version of the 
build side table.
-In our example, the query uses the processing-time notion, so a newly appended 
order would always be joined with the most recent version of `LatestRates` when 
executing the operation.
+**Note** The reason why the `FOR SYSTEM_TIME AS OF` syntax used in temporal 
join with latest version of any table/view is not support is only the semantic 
consideration, because the join processing for left stream doesn't wait for the 
complete snapshot of temporal table, this may mislead users in production 
environment. The processing-time temporal join by temporal table function also 
exists same semantic problem, but it has been alive for a long time, thus we 
support it from the perspec [...]
 
 The result is not deterministic for processing-time.
 The processing-time temporal join is most often used to enrich the stream with 
an external table (i.e., dimension table).
@@ -268,6 +270,29 @@ The processing-time temporal join is most often used to 
enrich the stream with a
 In contrast to [regular joins](#regular-joins), the previous temporal table 
results will not be affected despite the changes on the build side.
 Compared to [interval joins](#interval-joins), temporal table joins do not 
define a time window within which the records join, i.e., old rows are not 
stored in state.
 
+### Temporal Table Function Join
+
+The syntax to join a table with a [temporal table function]({{< ref 
"docs/dev/table/concepts/temporal_table_function" >}}) is the same as in Join 
with [Table Function](#table-function).
+
+Note: Currently only inner join and left outer join with temporal tables are 
supported.
+
+Assuming Rates is a temporal table function, the join can be expressed in SQL 
as follows:
+
+```sql
+SELECT
+  o_amount, r_rate
+FROM
+  Orders,
+  LATERAL TABLE (Rates(o_proctime))
+WHERE
+  r_currency = o_currency
+```
+
+The main difference between above Temporal Table DDL and Temporal Table 
Function are:
+
+- The temporal table DDL can be defined in SQL but temporal table function can 
not;
+- Both temporal table DDL and temporal table function support temporal join 
versioned table, but only temporal table function can temporal join the latest 
version of any table/view.
+
 Lookup Join
 --------------
 
diff --git a/docs/content/docs/dev/table/concepts/legacy.md 
b/docs/content/docs/dev/table/concepts/temporal_table_function.md
similarity index 82%
rename from docs/content/docs/dev/table/concepts/legacy.md
rename to docs/content/docs/dev/table/concepts/temporal_table_function.md
index ff5c3b6..ac6665d 100644
--- a/docs/content/docs/dev/table/concepts/legacy.md
+++ b/docs/content/docs/dev/table/concepts/temporal_table_function.md
@@ -1,5 +1,5 @@
 ---
-title: "Legacy Features"
+title: "Temporal Table Function"
 weight: 1002
 type: docs
 aliases:
@@ -24,22 +24,17 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-# Legacy Features
-
-As Flink SQL has matured there are some features that have been replaced with 
more modern and better functioning substitutes.
-These legacy features remain documented here for those users that have not yet 
or are unable to, upgrade to the more modern variant.
-
 # Temporal Table Function
 
-The temporal table function is the legacy way of defining something akin to a 
[versioned table]({{< ref "docs/dev/table/concepts/versioned_tables" >}})
-that can be used in a temporal table join.
-Please define temporal joins using [versioned tables]({{< ref 
"docs/dev/table/concepts/versioned_tables" >}}) in new queries.
+A Temporal table function provides access to the version of a temporal table 
at a specific point in time.
+In order to access the data in a temporal table, one must pass a [time 
attribute]({{<ref "docs/dev/table/concepts/time_attributes">}}) that determines 
the version of the table that will be returned.
+Flink uses the SQL syntax of [table functions]({{<ref 
"docs/dev/table/functions/udfs" >}}#table-functions) to provide a way to 
express it.
 
 Unlike a versioned table, temporal table functions can only be defined on top 
of append-only streams 
 &mdash; it does not support changelog inputs.
 Additionally, a temporal table function cannot be defined in pure SQL DDL. 
-
-#### Defining a Temporal Table Function
+ 
+## Defining a Temporal Table Function
 
 Temporal table functions can be defined on top of append-only streams using 
the [Table API]({{< ref "docs/dev/table/tableApi" >}}).
 The table is registered with one or more key columns, and a time attribute 
used for versioning.
@@ -83,9 +78,9 @@ tEnv.registerFunction("rates", rates)
 {{< /tab >}}
 {{< /tabs >}}
 
-#### Temporal Table Function Join
+## Temporal Table Function Join
 
-Once defined, a temporal table function is used as a standard [table 
function]({{< ref "/docs/dev/table/functions/udfs" >}}#table-functions).
+Once defined, a temporal table function is used as a standard [table 
function]({{< ref "docs/dev/table/functions/udfs" >}}#table-functions).
 Append-only tables (left input/probe side) can join with a temporal table 
(right input/build side),
 i.e., a table that changes over time and tracks its changes, to retrieve the 
value for a key as it was at a particular point in time.
 
diff --git a/docs/content/docs/dev/table/sql/queries/joins.md 
b/docs/content/docs/dev/table/sql/queries/joins.md
index d2fa42b..484ec1b 100644
--- a/docs/content/docs/dev/table/sql/queries/joins.md
+++ b/docs/content/docs/dev/table/sql/queries/joins.md
@@ -109,9 +109,12 @@ Since time attributes are quasi-monotonic increasing, 
Flink can remove old value
 Temporal Joins
 --------------
 
+A Temporal table is a table that evolves over time - otherwise known in Flink 
as a [dynamic table]({% link dev/table/streaming/dynamic_tables.md %}). Rows in 
a temporal table are associated with one or more temporal periods and all Flink 
tables are temporal(dynamic).
+The temporal table contains one or more versioned table snapshots, it can be a 
changing history table which tracks the changes(e.g. database changelog, 
contains all snapshots) or a changing dimensioned table which materializes the 
changes(e.g. database table which contains the latest snapshot). 
+
 ### Event Time Temporal Join
 
-Temporal joins allow joining against a [versioned table]({{< ref 
"docs/dev/table/concepts/versioned_tables" >}}).
+Event Time temporal joins allow joining against a [versioned table]({{< ref 
"docs/dev/table/concepts/versioned_tables" >}}).
 This means a table can be enriched with changing metadata and retrieve its 
value at a certain point in time. 
 
 Temporal joins take an arbitrary table (left input/probe site) and correlate 
each row to the corresponding row's relevant version in the versioned table 
(right input/build side). 
@@ -247,20 +250,19 @@ amount currency     rate   amount*rate
      2 Euro          116          232    <== arrived at time 10:52
 ```
 
-
-With the help of temporal table join, we can express such a query in SQL as:
+Currently, the `FOR SYSTEM_TIME AS OF` syntax used in temporal join with 
latest version of any view/table is not support yet, you can use temporal table 
function syntax as following:
 
 ```sql
 SELECT
-  o.amount, o.currency, r.rate, o.amount * r.rate
+  o_amount, r_rate
 FROM
-  Orders AS o
-  JOIN LatestRates FOR SYSTEM_TIME AS OF o.proctime AS r
-  ON r.currency = o.currency
+  Orders,
+  LATERAL TABLE (Rates(o_proctime))
+WHERE
+  r_currency = o_currency
 ```
 
-Each record from the probe side will be joined with the current version of the 
build side table.
-In our example, the query uses the processing-time notion, so a newly appended 
order would always be joined with the most recent version of `LatestRates` when 
executing the operation.
+**Note** The reason why the `FOR SYSTEM_TIME AS OF` syntax used in temporal 
join with latest version of any table/view is not support is only the semantic 
consideration, because the join processing for left stream doesn't wait for the 
complete snapshot of temporal table, this may mislead users in production 
environment. The processing-time temporal join by temporal table function also 
exists same semantic problem, but it has been alive for a long time, thus we 
support it from the perspec [...]
 
 The result is not deterministic for processing-time.
 The processing-time temporal join is most often used to enrich the stream with 
an external table (i.e., dimension table).
@@ -268,6 +270,29 @@ The processing-time temporal join is most often used to 
enrich the stream with a
 In contrast to [regular joins](#regular-joins), the previous temporal table 
results will not be affected despite the changes on the build side.
 Compared to [interval joins](#interval-joins), temporal table joins do not 
define a time window within which the records join, i.e., old rows are not 
stored in state.
 
+### Temporal Table Function Join
+
+The syntax to join a table with a [temporal table function]({{< ref 
"docs/dev/table/concepts/temporal_table_function" >}}) is the same as in Join 
with [Table Function](#table-function).
+
+Note: Currently only inner join and left outer join with temporal tables are 
supported.
+
+Assuming Rates is a temporal table function, the join can be expressed in SQL 
as follows:
+
+```sql
+SELECT
+  o_amount, r_rate
+FROM
+  Orders,
+  LATERAL TABLE (Rates(o_proctime))
+WHERE
+  r_currency = o_currency
+```
+
+The main difference between above Temporal Table DDL and Temporal Table 
Function are:
+
+- The temporal table DDL can be defined in SQL but temporal table function can 
not;
+- Both temporal table DDL and temporal table function support temporal join 
versioned table, but only temporal table function can temporal join the latest 
version of any table/view.
+
 Lookup Join
 --------------
 
diff --git a/docs/content/docs/dev/table/tableApi.md 
b/docs/content/docs/dev/table/tableApi.md
index d9684ed..cd4e177 100644
--- a/docs/content/docs/dev/table/tableApi.md
+++ b/docs/content/docs/dev/table/tableApi.md
@@ -1108,7 +1108,7 @@ result = joined_table.select(joined_table.a, 
joined_table.b, joined_table.s, joi
 {{< /tab >}}
 {{< /tabs >}}
 
-#### Join with Temporal TAble
+#### Join with Temporal Table
 
 Temporal tables are tables that track changes over time.
 

Reply via email to