You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by dw...@apache.org on 2020/06/09 13:52:55 UTC
[flink] 03/04: [FLINK-18195] Remove references to
Expressions.interval(Duration) from (java)docs
This is an automated email from the ASF dual-hosted git repository.
dwysakowicz pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 531ecc553a362bdf44bce2572ae064b3c856195e
Author: Dawid Wysakowicz <dw...@apache.org>
AuthorDate: Mon Jun 8 22:14:29 2020 +0200
[FLINK-18195] Remove references to Expressions.interval(Duration) from (java)docs
This closes #12535
---
docs/dev/table/streaming/time_attributes.md | 8 ++---
docs/dev/table/streaming/time_attributes.zh.md | 8 ++---
docs/dev/table/tableApi.md | 34 +++++++++++-----------
docs/dev/table/tableApi.zh.md | 34 +++++++++++-----------
.../org/apache/flink/table/api/Expressions.java | 5 ++--
5 files changed, 45 insertions(+), 44 deletions(-)
diff --git a/docs/dev/table/streaming/time_attributes.md b/docs/dev/table/streaming/time_attributes.md
index a94d0bd..0667d92 100644
--- a/docs/dev/table/streaming/time_attributes.md
+++ b/docs/dev/table/streaming/time_attributes.md
@@ -123,7 +123,7 @@ DataStream<Tuple2<String, String>> stream = ...;
Table table = tEnv.fromDataStream(stream, $("user_name"), $("data"), $("user_action_time").proctime()");
WindowedTable windowedTable = table.window(
- Tumble.over(interval(Duration.ofMinutes(10)))
+ Tumble.over(lit(10).minutes())
.on($("user_action_time"))
.as("userActionWindow"));
{% endhighlight %}
@@ -177,7 +177,7 @@ tEnv.registerTableSource("user_actions", new UserActionSource());
WindowedTable windowedTable = tEnv
.from("user_actions")
.window(Tumble
- .over(interval(Duration.ofMinutes(10)))
+ .over(lit(10).minutes())
.on($("user_action_time"))
.as("userActionWindow"));
{% endhighlight %}
@@ -285,7 +285,7 @@ Table table = tEnv.fromDataStream(stream, $("user_action_time").rowtime(), $("us
// Usage:
WindowedTable windowedTable = table.window(Tumble
- .over(interval(Duration.ofMinutes(10)))
+ .over(lit(10).minutes())
.on($("user_action_time"))
.as("userActionWindow"));
{% endhighlight %}
@@ -368,7 +368,7 @@ tEnv.registerTableSource("user_actions", new UserActionSource());
WindowedTable windowedTable = tEnv
.from("user_actions")
- .window(Tumble.over(interval(Duration.ofMinutes(10))).on($("user_action_time")).as("userActionWindow"));
+ .window(Tumble.over(lit(10).minutes()).on($("user_action_time")).as("userActionWindow"));
{% endhighlight %}
</div>
<div data-lang="scala" markdown="1">
diff --git a/docs/dev/table/streaming/time_attributes.zh.md b/docs/dev/table/streaming/time_attributes.zh.md
index 86050d5..9cbc164 100644
--- a/docs/dev/table/streaming/time_attributes.zh.md
+++ b/docs/dev/table/streaming/time_attributes.zh.md
@@ -123,7 +123,7 @@ DataStream<Tuple2<String, String>> stream = ...;
Table table = tEnv.fromDataStream(stream, $("user_name"), $("data"), $("user_action_time").proctime()");
WindowedTable windowedTable = table.window(
- Tumble.over(interval(Duration.ofMinutes(10)))
+ Tumble.over(lit(10).minutes())
.on($("user_action_time"))
.as("userActionWindow"));
{% endhighlight %}
@@ -177,7 +177,7 @@ tEnv.registerTableSource("user_actions", new UserActionSource());
WindowedTable windowedTable = tEnv
.from("user_actions")
.window(Tumble
- .over(interval(Duration.ofMinutes(10)))
+ .over(lit(10).minutes())
.on($("user_action_time"))
.as("userActionWindow"));
{% endhighlight %}
@@ -284,7 +284,7 @@ Table table = tEnv.fromDataStream(stream, $("user_action_time").rowtime(), $("us
// Usage:
WindowedTable windowedTable = table.window(Tumble
- .over(interval(Duration.ofMinutes(10)))
+ .over(lit(10).minutes())
.on($("user_action_time"))
.as("userActionWindow"));
{% endhighlight %}
@@ -365,7 +365,7 @@ tEnv.registerTableSource("user_actions", new UserActionSource());
WindowedTable windowedTable = tEnv
.from("user_actions")
- .window(Tumble.over(interval(Duration.ofMinutes(10))).on($("user_action_time")).as("userActionWindow"));
+ .window(Tumble.over(lit(10).minutes()).on($("user_action_time")).as("userActionWindow"));
{% endhighlight %}
</div>
<div data-lang="scala" markdown="1">
diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md
index bc88eb5..a4e1f5c 100644
--- a/docs/dev/table/tableApi.md
+++ b/docs/dev/table/tableApi.md
@@ -148,7 +148,7 @@ Table result = orders
$("c").isNotNull()
))
.select($("a").lowerCase().as("a"), $("b"), $("rowtime"))
- .window(Tumble.over(interval(Duration.ofHours(1))).on($("rowtime")).as("hourlyWindow"))
+ .window(Tumble.over(lit(1).hours()).on($("rowtime")).as("hourlyWindow"))
.groupBy($("hourlyWindow"), $("a"))
.select($("a"), $("hourlyWindow").end().as("hour"), $("b").avg().as("avgBillingAmount"));
{% endhighlight %}
@@ -764,7 +764,7 @@ Table result = orders.groupBy($("a")).select($("a"), $("b").sum().as("d"));
{% highlight java %}
Table orders = tableEnv.from("Orders");
Table result = orders
- .window(Tumble.over(interval(Duration.ofMinutes(5))).on($("rowtime")).as("w")) // define window
+ .window(Tumble.over(lit(5).minutes())).on($("rowtime")).as("w")) // define window
.groupBy($("a"), $("w")) // group by key and window
// access window properties and aggregate
.select(
@@ -823,7 +823,7 @@ Table groupByDistinctResult = orders
// Distinct aggregation on time window group by
Table groupByWindowDistinctResult = orders
.window(Tumble
- .over(interval(Duration.ofMinutes(5)))
+ .over(lit(5).minutes()))
.on($("rowtime"))
.as("w")
)
@@ -1185,8 +1185,8 @@ Table result = left.join(right)
.where(
and(
$("a").isEqual($("d")),
- $("ltime").isGreaterEqual($("rtime").minus(interval(Duration.ofMinutes(5)))),
- $("ltime").isLess($("rtime").plus(interval(Duration.ofMinutes(10))))
+ $("ltime").isGreaterEqual($("rtime").minus(lit(5).minutes())),
+ $("ltime").isLess($("rtime").plus(lit(10).minutes()))
))
.select($("a"), $("b"), $("e"), $("ltime"));
{% endhighlight %}
@@ -2273,10 +2273,10 @@ Tumbling windows are defined by using the `Tumble` class as follows:
<div data-lang="java" markdown="1">
{% highlight java %}
// Tumbling Event-time Window
-.window(Tumble.over(interval(Duration.ofMinutes(10))).on($("rowtime")).as("w"));
+.window(Tumble.over(lit(10).minutes()).on($("rowtime")).as("w"));
// Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
-.window(Tumble.over(interval(Duration.ofMinutes(10))).on($("proctime")).as("w"));
+.window(Tumble.over(lit(10).minutes()).on($("proctime")).as("w"));
// Tumbling Row-count Window (assuming a processing-time attribute "proctime")
.window(Tumble.over(rowInterval(10)).on($("proctime")).as("w"));
@@ -2348,14 +2348,14 @@ Sliding windows are defined by using the `Slide` class as follows:
<div data-lang="java" markdown="1">
{% highlight java %}
// Sliding Event-time Window
-.window(Slide.over(interval(Duration.ofMinutes(10)))
- .every(interval(Duration.ofMinutes(5)))
+.window(Slide.over(lit(10).minutes())
+ .every(lit(5).minutes())
.on($("rowtime"))
.as("w"));
// Sliding Processing-time window (assuming a processing-time attribute "proctime")
-.window(Slide.over(interval(Duration.ofMinutes(10)))
- .every(interval(Duration.ofMinutes(5)))
+.window(Slide.over(lit(10).minutes())
+ .every(lit(5).minutes())
.on($("proctime"))
.as("w"));
@@ -2425,10 +2425,10 @@ A session window is defined by using the `Session` class as follows:
<div data-lang="java" markdown="1">
{% highlight java %}
// Session Event-time Window
-.window(Session.withGap(interval(Duration.ofMinutes(10))).on($("rowtime")).as("w"));
+.window(Session.withGap(lit(10).minutes()).on($("rowtime")).as("w"));
// Session Processing-time Window (assuming a processing-time attribute "proctime")
-.window(Session.withGap(interval(Duration.ofMinutes(10))).on($("proctime")).as("w"));
+.window(Session.withGap(lit(10).minutes()).on($("proctime")).as("w"));
{% endhighlight %}
</div>
@@ -2614,10 +2614,10 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov
<div data-lang="java" markdown="1">
{% highlight java %}
// Bounded Event-time over window (assuming an event-time attribute "rowtime")
-.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(interval(Duration.ofMinutes(1))).as("w"))
+.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(lit(1).minutes()).as("w"))
// Bounded Processing-time over window (assuming a processing-time attribute "proctime")
-.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(interval(Duration.ofMinutes(1))).as("w"))
+.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(lit(1).minutes()).as("w"))
// Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(rowInterval(10)).as("w"))
@@ -2805,7 +2805,7 @@ AggregateFunction myAggFunc = new MyMinMax();
tableEnv.registerFunction("myAggFunc", myAggFunc);
Table table = input
- .window(Tumble.over(interval(Duration.ofMinutes(5)))
+ .window(Tumble.over(lit(5).minutes())
.on($("rowtime"))
.as("w")) // define window
.groupBy($("key"), $("w")) // group by key and window
@@ -2897,7 +2897,7 @@ Table result = orders
tableEnv.registerFunction("top2", new Top2());
Table orders = tableEnv.from("Orders");
Table result = orders
- .window(Tumble.over(interval(Duration.ofMinutes(5)))
+ .window(Tumble.over(lit(5).minutes())
.on($("rowtime"))
.as("w")) // define window
.groupBy($("a"), $("w")) // group by key and window
diff --git a/docs/dev/table/tableApi.zh.md b/docs/dev/table/tableApi.zh.md
index ee9c6d8..0ac1d80 100644
--- a/docs/dev/table/tableApi.zh.md
+++ b/docs/dev/table/tableApi.zh.md
@@ -148,7 +148,7 @@ Table result = orders
$("c").isNotNull()
))
.select($("a").lowerCase().as("a"), $("b"), $("rowtime"))
- .window(Tumble.over(interval(Duration.ofHours(1))).on($("rowtime")).as("hourlyWindow"))
+ .window(Tumble.over(lit(1).hours()).on($("rowtime")).as("hourlyWindow"))
.groupBy($("hourlyWindow"), $("a"))
.select($("a"), $("hourlyWindow").end().as("hour"), $("b").avg().as("avgBillingAmount"));
{% endhighlight %}
@@ -764,7 +764,7 @@ Table result = orders.groupBy($("a")).select($("a"), $("b").sum().as("d"));
{% highlight java %}
Table orders = tableEnv.from("Orders");
Table result = orders
- .window(Tumble.over(interval(Duration.ofMinutes(5))).on($("rowtime")).as("w")) // define window
+ .window(Tumble.over(lit(5).minutes()).on($("rowtime")).as("w")) // define window
.groupBy($("a"), $("w")) // group by key and window
// access window properties and aggregate
.select(
@@ -823,7 +823,7 @@ Table groupByDistinctResult = orders
// Distinct aggregation on time window group by
Table groupByWindowDistinctResult = orders
.window(Tumble
- .over(interval(Duration.ofMinutes(5)))
+ .over(lit(5).minutes())
.on($("rowtime"))
.as("w")
)
@@ -1184,8 +1184,8 @@ Table result = left.join(right)
.where(
and(
$("a").isEqual($("d")),
- $("ltime").isGreaterEqual($("rtime").minus(interval(Duration.ofMinutes(5)))),
- $("ltime").isLess($("rtime").plus(interval(Duration.ofMinutes(10))))
+ $("ltime").isGreaterEqual($("rtime").minus(lit(5).minutes())),
+ $("ltime").isLess($("rtime").plus(lit(10).minutes()))
))
.select($("a"), $("b"), $("e"), $("ltime"));
{% endhighlight %}
@@ -2272,10 +2272,10 @@ Tumbling windows are defined by using the `Tumble` class as follows:
<div data-lang="java" markdown="1">
{% highlight java %}
// Tumbling Event-time Window
-.window(Tumble.over(interval(Duration.ofMinutes(10))).on($("rowtime")).as("w"));
+.window(Tumble.over(lit(10).minutes()).on($("rowtime")).as("w"));
// Tumbling Processing-time Window (assuming a processing-time attribute "proctime")
-.window(Tumble.over(interval(Duration.ofMinutes(10))).on($("proctime")).as("w"));
+.window(Tumble.over(lit(10).minutes()).on($("proctime")).as("w"));
// Tumbling Row-count Window (assuming a processing-time attribute "proctime")
.window(Tumble.over(rowInterval(10)).on($("proctime")).as("w"));
@@ -2347,14 +2347,14 @@ Sliding windows are defined by using the `Slide` class as follows:
<div data-lang="java" markdown="1">
{% highlight java %}
// Sliding Event-time Window
-.window(Slide.over(interval(Duration.ofMinutes(10)))
- .every(interval(Duration.ofMinutes(5)))
+.window(Slide.over(lit(10).minutes())
+ .every(lit(5).minutes())
.on($("rowtime"))
.as("w"));
// Sliding Processing-time window (assuming a processing-time attribute "proctime")
-.window(Slide.over(interval(Duration.ofMinutes(10)))
- .every(interval(Duration.ofMinutes(5)))
+.window(Slide.over(lit(10).minutes())
+ .every(lit(5).minutes())
.on($("proctime"))
.as("w"));
@@ -2424,10 +2424,10 @@ A session window is defined by using the `Session` class as follows:
<div data-lang="java" markdown="1">
{% highlight java %}
// Session Event-time Window
-.window(Session.withGap(interval(Duration.ofMinutes(10))).on($("rowtime")).as("w"));
+.window(Session.withGap(lit(10).minutes()).on($("rowtime")).as("w"));
// Session Processing-time Window (assuming a processing-time attribute "proctime")
-.window(Session.withGap(interval(Duration.ofMinutes(10))).on($("proctime")).as("w"));
+.window(Session.withGap(lit(10).minutes()).on($("proctime")).as("w"));
{% endhighlight %}
</div>
@@ -2613,10 +2613,10 @@ The `OverWindow` defines a range of rows over which aggregates are computed. `Ov
<div data-lang="java" markdown="1">
{% highlight java %}
// Bounded Event-time over window (assuming an event-time attribute "rowtime")
-.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(interval(Duration.ofMinutes(1))).as("w"))
+.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(lit(1).minutes()).as("w"))
// Bounded Processing-time over window (assuming a processing-time attribute "proctime")
-.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(interval(Duration.ofMinutes(1))).as("w"))
+.window(Over.partitionBy($("a")).orderBy($("proctime")).preceding(lit(1).minutes()).as("w"))
// Bounded Event-time Row-count over window (assuming an event-time attribute "rowtime")
.window(Over.partitionBy($("a")).orderBy($("rowtime")).preceding(rowInterval(10)).as("w"))
@@ -2804,7 +2804,7 @@ AggregateFunction myAggFunc = new MyMinMax();
tableEnv.registerFunction("myAggFunc", myAggFunc);
Table table = input
- .window(Tumble.over(interval(Duration.ofMinutes(5)))
+ .window(Tumble.over(lit(5).minutes())
.on($("rowtime"))
.as("w")) // define window
.groupBy($("key"), $("w")) // group by key and window
@@ -2896,7 +2896,7 @@ Table result = orders
tableEnv.registerFunction("top2", new Top2());
Table orders = tableEnv.from("Orders");
Table result = orders
- .window(Tumble.over(interval(Duration.ofMinutes(5)))
+ .window(Tumble.over(lit(5).minutes())
.on($("rowtime"))
.as("w")) // define window
.groupBy($("a"), $("w")) // group by key and window
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
index 37f53d4..1493486 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
@@ -225,9 +225,10 @@ public final class Expressions {
* <pre>{@code
* temporalOverlaps(
* lit("2:55:00").toTime(),
- * interval(Duration.ofHour(1)),
+ * lit(1).hours(),
* lit("3:30:00").toTime(),
- * interval(Duration.ofHour(2))
+ * lit(2).hours()
+ * )
* }</pre>
* leads to true
*/