WencongLiu commented on code in PR #23362:
URL: https://github.com/apache/flink/pull/23362#discussion_r1360393473


##########
docs/content/docs/dev/datastream/dataset_migration.md:
##########
@@ -0,0 +1,758 @@
+---
+title: "How To Migrate From DataSet to DataStream"
+weight: 302
+type: docs
+---
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+# How to Migrate from DataSet to DataStream
+
+The DataSet API has been formally deprecated and will no longer receive active 
maintenance and support. It will be removed in the
+Flink 2.0 version. Flink users are recommended to migrate from the DataSet API 
to the DataStream API, Table API and SQL for their 
+data processing requirements. 
+
+For the most of DataSet APIs, the users can utilize the DataStream API to get 
the same calculation result in the batch jobs. However,
+different DataSet API can be implemented by DataStream API with various 
difference on semantic and behavior. All DataSet APIs can be
+categorized into four types:
+
+Category 1: These DataSet APIs can be implemented by DataStream APIs with same 
semantic and same calculation behavior.
+
+Category 2: These DataSet APIs can be implemented by DataStream APIs with 
different semantic but same calculation behavior. This will 
+make the job code more complex.
+
+Category 3: These DataSet APIs can be implemented by DataStream APIs with 
different semantic and different calculation behavior. This 
+will involve additional computation and shuffle costs.
+
+Category 4: These DataSet APIs are not supported by DataStream APIs.
+
+The subsequent sections will first introduce how to set the execution 
environment and provide detailed explanations on how to implement 
+each category of DataSet APIs using the DataStream APIs, highlighting the 
specific considerations and challenges associated with each 
+category.
+
+
+## Setting the execution environment
+
+To execute a DataSet pipeline by DataStream API, we should first start by 
replacing ExecutionEnvironment with StreamExecutionEnvironment.
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">// Create the execution 
environment
+            ExecutionEnvironment.getExecutionEnvironment();
+            // Create the local execution environment
+            ExecutionEnvironment.createLocalEnvironment();
+            // Create the collection environment
+            new CollectionEnvironment();
+            // Create the remote environment
+            ExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">// Create the execution 
environment
+            StreamExecutionEnvironment.getExecutionEnvironment();
+            // Create the local execution environment
+            StreamExecutionEnvironment.createLocalEnvironment();
+            // The collection environment is not supported.
+            // Create the remote environment
+            StreamExecutionEnvironment.createRemoteEnvironment(host, port, 
jarFile);
+        </code>
+    </td>
+  </tr>
+</table>
+
+
+As the source of DataSet is always bounded, the execution mode must be set to 
RuntimeMode.BATCH to make Flink execute in batch mode.
+
+```java
+StreamExecutionEnvironment executionEnvironment = 
StreamExecutionEnvironment.getExecutionEnvironment();
+executionEnvironment.setRuntimeMode(RuntimeExecutionMode.BATCH);
+```
+
+## Using the streaming sources and sinks
+
+Sources: The DataStream API uses `DataStreamSource` to read records from 
external system, while the DataSet API uses the
+`DataSource`.
+
+Sinks: The DataStream API uses the implementations of `SinkFunction` and 
`Sink` to write records to external system, while the
+DataSet API uses the `FileOutputFormat`.
+
+If you are looking for pre-defined source and sink connectors of DataStream, 
please check the [Connector Docs]({{< ref "docs/connectors/datastream/overview" 
>}})
+
+
+## Implement the DataSet API by DataStream
+
+#### Category 1
+
+For Category 1, the usage of the API in DataStream is almost identical to that 
in DataSet. This means that implementing these 
+DataSet APIs by the DataStream API is relatively straightforward and does not 
require significant modifications or complexity
+in the job code.
+
+### Map
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">dataSet.map(new MapFunction(){
+    // implement user-defined map logic
+});
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">dataStream.map(new MapFunction(){
+    // implement user-defined map logic
+});
+        </code>
+    </td>
+  </tr>
+</table>
+
+
+### FlatMap
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">dataSet.flatMap(new 
FlatMapFunction(){
+    // implement user-defined flatmap logic
+});
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">dataStream.flatMap(new 
FlatMapFunction(){
+    // implement user-defined flatmap logic
+});
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Filter
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">dataSet.filter(new 
FilterFunction(){
+    // implement user-defined filter logic
+});
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">dataStream.filter(new 
FilterFunction(){
+    // implement user-defined filter logic
+});
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Union
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">dataSet1.union(dataSet2);
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">dataStream1.union(dataStream2);
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Rebalance
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">dataSet.rebalance();
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">dataStream.rebalance();
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Project
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">DataSet&ltTuple3&ltInteger, 
Double, String&gt&gt dataSet = // [...]
+dataSet.project(2,0);
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">DataStream&ltTuple3&ltInteger, 
Double, String&gt&gt dataStream = // [...]
+dataStream.project(2,0);
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Reduce on Grouped DataSet
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">DataSet&ltTuple2&ltString, 
Integer&gt&gt dataSet = // [...]
+dataSet.groupBy(value -> value.f0)
+        .reduce(new ReduceFunction(){
+            // implement user-defined reduce logic
+        });
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">DataStream&ltTuple2&ltString, 
Integer&gt&gt dataStream = // [...]
+dataStream.keyBy(value -> value.f0)
+        .reduce(new ReduceFunction(){
+            // implement user-defined reduce logic
+        });
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Aggregate on Grouped DataSet
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">DataSet&ltTuple2&ltString, 
Integer&gt&gt dataSet = // [...]
+// compute sum of the second field
+dataSet.groupBy(value -> value.f0).aggregate(SUM, 1);
+// compute min of the second field
+dataSet.groupBy(value -> value.f0).aggregate(MIN, 1);
+// compute max of the second field
+dataSet.groupBy(value -> value.f0).aggregate(MAX, 1);
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">DataStream&ltTuple2&ltString, 
Integer&gt&gt dataStream = // [...]
+// compute sum of the second field
+dataStream.keyBy(value -> value.f0).sum(1);
+// compute min of the second field
+dataStream.keyBy(value -> value.f0).min(1);
+// compute max of the second field
+dataStream.keyBy(value -> value.f0).max(1);
+        </code>
+    </td>
+  </tr>
+</table>
+
+#### Category 2
+
+For category 2, these DataSet APIs can be implemented by DataStream APIs with 
different semantic but same calculation behavior.
+The developers need to adapt their code to accommodate these variations, which 
introduces additional complexity.
+
+### Distinct
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">DataSet&ltInteger&gt dataSet = // 
[...]
+dataSet.distinct();
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">DataStream&ltInteger&gt 
dataStream = // [...]
+dataStream
+        .keyBy(value -> value)
+        .reduce((value1, value2) -> value1);
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Hash-Partition
+
+<table>
+  <tr>
+    <th>DataSet</th>
+    <th>DataStream</th>
+  </tr>
+  <tr>
+    <td>
+        <code style="white-space: pre-line;">DataSet&ltTuple2&ltString, 
Integer&gt&gt dataSet = // [...]
+dataSet.partitionByHash(value -> value.f0);
+        </code>
+    </td>
+    <td>
+        <code style="white-space: pre-line;">DataStream&ltTuple2&ltString, 
Integer&gt&gt dataStream = // [...]
+// partition by the hashcode of key
+dataStream.partitionCustom((key, numSubpartition) -> key.hashCode() % 
numSubpartition, value -> value.f0);
+        </code>
+    </td>
+  </tr>
+</table>
+
+### Reduce on Full DataSet
+
+If developers want to compute data of full datastream, `GlobalWindow` could be 
used to collect all records of datastream.
+However, a special trigger is also required to trigger the computation of 
`GlobalWindow` at the end of its inputs. Here is an example 
+code snippet of the trigger.
+
+```java
+public class EOFTrigger extends Trigger<Object, GlobalWindow> {

Review Comment:
   The section about "Sink and Source" is updated.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to