klion26 commented on a change in pull request #12665:
URL: https://github.com/apache/flink/pull/12665#discussion_r457070589



##########
File path: docs/dev/event_timestamp_extractors.zh.md
##########
@@ -25,83 +25,54 @@ under the License.
 * toc
 {:toc}
 
-As described in [timestamps and watermark handling]({{ site.baseurl 
}}/dev/event_timestamps_watermarks.html),
-Flink provides abstractions that allow the programmer to assign their own 
timestamps and emit their own watermarks. More specifically,
-one can do so by implementing one of the `AssignerWithPeriodicWatermarks` and 
`AssignerWithPunctuatedWatermarks` interfaces, depending
-on the use case. In a nutshell, the first will emit watermarks periodically, 
while the second does so based on some property of
-the incoming records, e.g. whenever a special element is encountered in the 
stream.
+如[生成 Watermark]({%link dev/event_timestamps_watermarks.zh.md %}) 小节中所述,Flink 
提供的抽象方法可以允许用户自己去定义时间戳分配方式和 watermark 生成的方式。你可以通过实现 `WatermarkGenerator` 
接口来实现上述功能。
 
-In order to further ease the programming effort for such tasks, Flink comes 
with some pre-implemented timestamp assigners.
-This section provides a list of them. Apart from their out-of-the-box 
functionality, their implementation can serve as an example
-for custom implementations.
+为了进一步简化此类任务的编程工作,Flink 框架预设了一些时间戳分配器。本节后续内容有举例。除了开箱即用的已有实现外,其还可以作为自定义实现的示例以供参考。
 
-### **Assigners with ascending timestamps**
+<a name="monotonously-increasing-timestamps"></a>
 
-The simplest special case for *periodic* watermark generation is the case 
where timestamps seen by a given source task
-occur in ascending order. In that case, the current timestamp can always act 
as a watermark, because no earlier timestamps will
-arrive.
+## 单调递增时间戳分配器
 
-Note that it is only necessary that timestamps are ascending *per parallel 
data source task*. For example, if
-in a specific setup one Kafka partition is read by one parallel data source 
instance, then it is only necessary that
-timestamps are ascending within each Kafka partition. Flink's watermark 
merging mechanism will generate correct
-watermarks whenever parallel streams are shuffled, unioned, connected, or 
merged.
+*周期性* watermark 生成方式的一个最简单特例就是你给定的数据源中数据的时间戳升序出现。在这种情况下,当前时间戳就可以充当 
watermark,因为后续到达数据的时间戳不会比当前的小。
+
+注意:在 Flink 应用程序中,如果是并行数据源,则只要求并行数据源中的每个*单分区数据源任务*时间戳递增。例如,设置每一个并行数据源实例都只读取一个 
Kafka 分区,则时间戳只需在每个 Kafka 分区内递增即可。Flink 的 watermark 
合并机制会在并行数据流进行分发(shuffle)、联合(union)、连接(connect)或合并(merge)时生成正确的 watermark。
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-DataStream<MyEvent> stream = ...
-
-DataStream<MyEvent> withTimestampsAndWatermarks =
-    stream.assignTimestampsAndWatermarks(new 
AscendingTimestampExtractor<MyEvent>() {
-
-        @Override
-        public long extractAscendingTimestamp(MyEvent element) {
-            return element.getCreationTime();
-        }
-});
+WatermarkStrategies
+        .<MyType>forMonotonousTimestamps()
+        .build();
 {% endhighlight %}
 </div>
 <div data-lang="scala" markdown="1">
 {% highlight scala %}
-val stream: DataStream[MyEvent] = ...
-
-val withTimestampsAndWatermarks = stream.assignAscendingTimestamps( 
_.getCreationTime )
+WatermarkStrategies
+  .forMonotonousTimestamps[MyType]()
+  .build()
 {% endhighlight %}
 </div>
 </div>
 
-### **Assigners allowing a fixed amount of lateness**
+<a name="fixed-amount-of-lateness"></a>
 
-Another example of periodic watermark generation is when the watermark lags 
behind the maximum (event-time) timestamp
-seen in the stream by a fixed amount of time. This case covers scenarios where 
the maximum lateness that can be encountered in a
-stream is known in advance, e.g. when creating a custom source containing 
elements with timestamps spread within a fixed period of
-time for testing. For these cases, Flink provides the 
`BoundedOutOfOrdernessTimestampExtractor` which takes as an argument
-the `maxOutOfOrderness`, i.e. the maximum amount of time an element is allowed 
to be late before being ignored when computing the
-final result for the given window. Lateness corresponds to the result of `t - 
t_w`, where `t` is the (event-time) timestamp of an
-element, and `t_w` that of the previous watermark. If `lateness > 0` then the 
element is considered late and is, by default, ignored when computing
-the result of the job for its corresponding window. See the documentation 
about [allowed lateness]({{ site.baseurl 
}}/dev/stream/operators/windows.html#allowed-lateness)
-for more information about working with late elements.
+## 数据之间存在最大固定延迟的时间戳分配器
+
+另一个周期性 watermark 生成的典型例子是,watermark 
滞后于数据流中最大(事件时间)时间戳一个固定的时间量。该示例可以覆盖的场景是你预先知道数据流中的数据可能遇到的最大延迟,例如,在测试场景下创建了一个自定义数据源,并且这个数据源的产生的数据的时间戳在一个固定范围之内。Flink
 针对上述场景提供了 `boundedOutfordernessWatermarks` 生成器,该生成器将 `maxOutOfOrderness` 
作为参数,该参数代表在计算给定窗口的结果时,允许元素被忽略计算之前延迟到达的最长时间。其中延迟时长就等于 `t_w - t` ,其中 `t` 
代表元素的(事件时间)时间戳,`t_w` 代表前一个 watermark 对应的(事件时间)时间戳。如果 `lateness > 
0`,则认为该元素迟到了,并且在计算相应窗口的结果时默认会被忽略。有关使用延迟元素的详细内容,请参阅有关[允许延迟]({% link d
 ev/stream/operators/windows.md %})的文档。
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-DataStream<MyEvent> stream = ...
-
-DataStream<MyEvent> withTimestampsAndWatermarks =
-    stream.assignTimestampsAndWatermarks(new 
BoundedOutOfOrdernessTimestampExtractor<MyEvent>(Time.seconds(10)) {
-
-        @Override
-        public long extractTimestamp(MyEvent element) {
-            return element.getCreationTime();
-        }
-});
+WatermarkStrategies

Review comment:
       这个代码和现在 英文 版的貌似不一样,这里可以再确认下。我这里看到的英文版是
   ```
   WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(10));
   ```

##########
File path: docs/dev/event_timestamps_watermarks.zh.md
##########
@@ -22,115 +22,77 @@ specific language governing permissions and limitations
 under the License.
 -->
 
+在本节中,你将了解 Flink 中用于处理**事件时间**的时间戳和 watermark 相关的 
API。有关*事件时间*,*处理时间*和*摄取时间*的介绍,请参阅[事件时间概览]({{ site.baseurl 
}}/zh/dev/event_time.html)小节。
+
 * toc
 {:toc}
 
+## Watermark 策略简介
 
-This section is relevant for programs running on **event time**. For an 
introduction to *event time*,
-*processing time*, and *ingestion time*, please refer to the [introduction to 
event time]({{ site.baseurl }}/dev/event_time.html).
-
-To work with *event time*, streaming programs need to set the *time 
characteristic* accordingly.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-{% endhighlight %}
-</div>
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val env = StreamExecutionEnvironment.getExecutionEnvironment
-env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-{% endhighlight %}
-</div>
-<div data-lang="python" markdown="1">
-{% highlight python %}
-env = StreamExecutionEnvironment.get_execution_environment()
-env.set_stream_time_characteristic(TimeCharacteristic.EventTime)
-{% endhighlight %}
-</div>
-</div>
-
-## Assigning Timestamps
-
-In order to work with *event time*, Flink needs to know the events' 
*timestamps*, meaning each element in the
-stream needs to have its event timestamp *assigned*. This is usually done by 
accessing/extracting the
-timestamp from some field in the element.
+为了使用*事件时间*语义,Flink 应用程序需要知道事件*时间戳*对应的字段,意味着数据流中的每个元素都需要拥有*可分配*的事件时间戳。其通常通过使用 
`TimestampAssigner` API 从元素中的某个字段去访问/提取时间戳。
 
-Timestamp assignment goes hand-in-hand with generating watermarks, which tell 
the system about
-progress in event time.
+时间戳的分配与 watermark 的生成是齐头并进的,其可以告诉 Flink 应用程序事件时间的进度。其可以通过指定 
`WatermarkGenerator` 来配置 watermark 的生成方式。
 
-There are two ways to assign timestamps and generate watermarks:
+使用 Flink API 时需要设置一个同时包含 `TimestampAssigner` 和 `WatermarkGenerator` 的 
`WatermarkStrategy`。`WatermarkStrategies` 工具类中也提供了许多常用的 watermark 
策略,并且用户也可以在某些必要场景下构建自己的 watermark 策略。WatermarkStrategy 接口如下:
 
-  1. Directly in the data stream source
-  2. Via a timestamp assigner / watermark generator: in Flink, timestamp 
assigners also define the watermarks to be emitted
-
-<span class="label label-danger">Attention</span> Both timestamps and 
watermarks are specified as
-milliseconds since the Java epoch of 1970-01-01T00:00:00Z.
-
-### Source Functions with Timestamps and Watermarks
+{% highlight java %}
+public interface WatermarkStrategy<T> extends TimestampAssignerSupplier<T>, 
WatermarkGeneratorSupplier<T>{
 
-Stream sources can directly assign timestamps to the elements they produce, 
and they can also emit watermarks.
-When this is done, no timestamp assigner is needed.
-Note that if a timestamp assigner is used, any timestamps and watermarks 
provided by the source will be overwritten.
+    /**
+     * Instantiates a {@link TimestampAssigner} for assigning timestamps 
according to this

Review comment:
       这里我没有明显的偏好。我的想法是翻译之后,其他人看到会更容易理解一些,你觉得呢?

##########
File path: docs/dev/event_timestamp_extractors.zh.md
##########
@@ -25,83 +25,54 @@ under the License.
 * toc
 {:toc}
 
-As described in [timestamps and watermark handling]({{ site.baseurl 
}}/dev/event_timestamps_watermarks.html),
-Flink provides abstractions that allow the programmer to assign their own 
timestamps and emit their own watermarks. More specifically,
-one can do so by implementing one of the `AssignerWithPeriodicWatermarks` and 
`AssignerWithPunctuatedWatermarks` interfaces, depending
-on the use case. In a nutshell, the first will emit watermarks periodically, 
while the second does so based on some property of
-the incoming records, e.g. whenever a special element is encountered in the 
stream.
+如[生成 Watermark]({%link dev/event_timestamps_watermarks.zh.md %}) 小节中所述,Flink 
提供的抽象方法可以允许用户自己去定义时间戳分配方式和 watermark 生成的方式。你可以通过实现 `WatermarkGenerator` 
接口来实现上述功能。
 
-In order to further ease the programming effort for such tasks, Flink comes 
with some pre-implemented timestamp assigners.
-This section provides a list of them. Apart from their out-of-the-box 
functionality, their implementation can serve as an example
-for custom implementations.
+为了进一步简化此类任务的编程工作,Flink 框架预设了一些时间戳分配器。本节后续内容有举例。除了开箱即用的已有实现外,其还可以作为自定义实现的示例以供参考。
 
-### **Assigners with ascending timestamps**
+<a name="monotonously-increasing-timestamps"></a>
 
-The simplest special case for *periodic* watermark generation is the case 
where timestamps seen by a given source task
-occur in ascending order. In that case, the current timestamp can always act 
as a watermark, because no earlier timestamps will
-arrive.
+## 单调递增时间戳分配器
 
-Note that it is only necessary that timestamps are ascending *per parallel 
data source task*. For example, if
-in a specific setup one Kafka partition is read by one parallel data source 
instance, then it is only necessary that
-timestamps are ascending within each Kafka partition. Flink's watermark 
merging mechanism will generate correct
-watermarks whenever parallel streams are shuffled, unioned, connected, or 
merged.
+*周期性* watermark 生成方式的一个最简单特例就是你给定的数据源中数据的时间戳升序出现。在这种情况下,当前时间戳就可以充当 
watermark,因为后续到达数据的时间戳不会比当前的小。
+
+注意:在 Flink 应用程序中,如果是并行数据源,则只要求并行数据源中的每个*单分区数据源任务*时间戳递增。例如,设置每一个并行数据源实例都只读取一个 
Kafka 分区,则时间戳只需在每个 Kafka 分区内递增即可。Flink 的 watermark 
合并机制会在并行数据流进行分发(shuffle)、联合(union)、连接(connect)或合并(merge)时生成正确的 watermark。
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-DataStream<MyEvent> stream = ...
-
-DataStream<MyEvent> withTimestampsAndWatermarks =
-    stream.assignTimestampsAndWatermarks(new 
AscendingTimestampExtractor<MyEvent>() {
-
-        @Override
-        public long extractAscendingTimestamp(MyEvent element) {
-            return element.getCreationTime();
-        }
-});
+WatermarkStrategies
+        .<MyType>forMonotonousTimestamps()
+        .build();
 {% endhighlight %}
 </div>
 <div data-lang="scala" markdown="1">
 {% highlight scala %}
-val stream: DataStream[MyEvent] = ...
-
-val withTimestampsAndWatermarks = stream.assignAscendingTimestamps( 
_.getCreationTime )
+WatermarkStrategies
+  .forMonotonousTimestamps[MyType]()
+  .build()
 {% endhighlight %}
 </div>
 </div>
 
-### **Assigners allowing a fixed amount of lateness**
+<a name="fixed-amount-of-lateness"></a>
 
-Another example of periodic watermark generation is when the watermark lags 
behind the maximum (event-time) timestamp
-seen in the stream by a fixed amount of time. This case covers scenarios where 
the maximum lateness that can be encountered in a
-stream is known in advance, e.g. when creating a custom source containing 
elements with timestamps spread within a fixed period of
-time for testing. For these cases, Flink provides the 
`BoundedOutOfOrdernessTimestampExtractor` which takes as an argument
-the `maxOutOfOrderness`, i.e. the maximum amount of time an element is allowed 
to be late before being ignored when computing the
-final result for the given window. Lateness corresponds to the result of `t - 
t_w`, where `t` is the (event-time) timestamp of an
-element, and `t_w` that of the previous watermark. If `lateness > 0` then the 
element is considered late and is, by default, ignored when computing
-the result of the job for its corresponding window. See the documentation 
about [allowed lateness]({{ site.baseurl 
}}/dev/stream/operators/windows.html#allowed-lateness)
-for more information about working with late elements.
+## 数据之间存在最大固定延迟的时间戳分配器
+
+另一个周期性 watermark 生成的典型例子是,watermark 
滞后于数据流中最大(事件时间)时间戳一个固定的时间量。该示例可以覆盖的场景是你预先知道数据流中的数据可能遇到的最大延迟,例如,在测试场景下创建了一个自定义数据源,并且这个数据源的产生的数据的时间戳在一个固定范围之内。Flink
 针对上述场景提供了 `boundedOutfordernessWatermarks` 生成器,该生成器将 `maxOutOfOrderness` 
作为参数,该参数代表在计算给定窗口的结果时,允许元素被忽略计算之前延迟到达的最长时间。其中延迟时长就等于 `t_w - t` ,其中 `t` 
代表元素的(事件时间)时间戳,`t_w` 代表前一个 watermark 对应的(事件时间)时间戳。如果 `lateness > 
0`,则认为该元素迟到了,并且在计算相应窗口的结果时默认会被忽略。有关使用延迟元素的详细内容,请参阅有关[允许延迟]({% link d
 ev/stream/operators/windows.md %})的文档。

Review comment:
       这里 `允许延迟` 的链接少了一个 `#allowed-lateness` 现在是链接到 window 页面,这里希望链接到对应的小节




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to