Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4365#discussion_r128245611
  
    --- Diff: docs/dev/table/streaming.md ---
    @@ -22,21 +22,166 @@ specific language governing permissions and limitations
     under the License.
     -->
     
    -**TO BE DONE:** Intro
    +Flink's [Table API](tableApi.html) and [SQL support](sql.html) are unified 
APIs for batch and stream processing. This means that Table API and SQL queries 
have the same semantics regardless whether their input is bounded batch input 
or unbounded stream input. Because the relational algebra and SQL were 
originally designed for batch processing, relational queries on unbounded 
streaming input are not as well understood as relational queries on bounded 
batch input. 
    +
    +On this page, we explain concepts, practical limitations, and 
stream-specific configuration parameters of Flink's relational APIs on 
streaming data. 
     
     * This will be replaced by the TOC
     {:toc}
     
    -Dynamic Table
    --------------
    +Relational Queries on Data Streams
    +----------------------------------
    +
    +SQL and the relational algebra have not been designed with streaming data 
in mind. As a consequence, there are few conceptual gaps between relational 
algebra (and SQL) and stream processing.
    +
    +<table class="table table-bordered">
    +   <tr>
    +           <th>Relational Algebra / SQL</th>
    +           <th>Stream Processing</th>
    +   </tr>
    +   <tr>
    +           <td>Relations (or tables) are bounded (multi-)sets of 
tuples.</td>
    +           <td>A stream is an infinite sequences of tuples.</td>
    +   </tr>
    +   <tr>
    +           <td>A query that is executed on batch data (e.g., a table in a 
relational database) has access to the complete input data.</td>
    +           <td>A streaming query cannot access all data when is started 
and has to "wait" for data to be streamed in.</td>
    +   </tr>
    +   <tr>
    +           <td>A batch query terminates after it produced a fixed sized 
result.</td>
    +           <td>A streaming query continuously updates its result based on 
the received records and never completes.</td>
    +   </tr>
    +</table>
    +
    +Despite these differences, processing streams with relational queries and 
SQL is not impossible. Advanced relational database systems offer a feature 
called *Materialized Views*. A materialized view is defined as a SQL query, 
just like a regular virtual view. In contrast to a virtual view, a materialized 
view caches the result of the query such that the query does not need to be 
evaluated when the view is accessed. A common challenge for caching is to 
prevent a cache from serving outdated results. A materialized view becomes 
outdated when the base tables of its definition query are modified. *Eager View 
Maintenance* is a technique to update materialized views and updates a 
materialized view as soon as its base tables are updated. 
    +
    +The connection between eager view maintenance and SQL queries on streams 
becomes obvious if we consider the following:
    +
    +- A database table is the result of a *stream* of `INSERT`, `UPDATE`, and 
`DELETE` DML statements, often called *changelog stream*.
    +- A materialized view is defined as a SQL query. In order to update the 
view, the query is continuously processes the changelog streams of the view's 
base relations.
    +- The materialized view is the result of the streaming SQL query.
    +
    +With these points in mind, we introduce Flink's concept of *Dynamic 
Tables* in the next section.
    +
    +Dynamic Tables &amp; Continuous Queries
    +---------------------------------------
    +
    +*Dynamic tables* are the core concept of Flink's Table API and SQL support 
for streaming data. In contrast to the static tables that represent batch data, 
dynamic table are changing over time. They can be queried like static batch 
tables. Querying a dynamic table yields a *Continuous Query*. A continuous 
query never terminates and produces a dynamic table as result. The query 
continuously updates its (dynamic) result table to reflect the changes on its 
input (dynamic) table. Essentially, a continuous query on a dynamic table is 
very similar to the definition query of a materialized view. 
    +
    +It is important to note that the result of a continuous query is always 
semantically equivalent to the result of the same query being executed in batch 
mode on a snapshot of the input tables.
    +
    +The following figure visualizes the relationship of streams, dynamic 
tables, and  continuous queries: 
    +
    +<center>
    +<img alt="Dynamic tables" src="{{ site.baseurl 
}}/fig/table-streaming/stream-query-stream.png" width="80%">
    +</center>
    +
    +1. A stream is converted into a dynamic table.
    +1. A continuous query is evaluated on the dynamic table yielding a new 
dynamic table.
    +1. The resulting dynamic table is converted back into a stream.
    +
    +**Note:** Dynamic tables are foremost a logical concept. Dynamic tables 
are not necessarily (fully) materialized during query execution.
    +
    +In the following, we will explain the concepts of dynamic tables and 
continuous queries with a stream of click events that have the following schema:
    +
    +```
    +[ 
    +  user:  VARCHAR,   // the name of the user
    +  cTime: TIMESTAMP, // the time when the URL was accessed
    +  url:   VARCHAR    // the URL that was accessed by the user
    +]
    +```
    +
    +### Defining a Table on a Stream
    +
    +In order to process a stream with a relational query, it has to be 
converted into a `Table`. Conceptually, each record of the stream is 
interpreted as an `INSERT` modification on the resulting table. Essentially, we 
are building a table from an `INSERT`-only changelog stream.
    +
    +The following figure visualizes how the stream of click event (left-hand 
side) is converted into a table (right-hand side). The resulting table is 
continuously growing as more records of the click stream are inserted.
    +
    +<center>
    +<img alt="Append mode" src="{{ site.baseurl 
}}/fig/table-streaming/append-mode.png" width="60%">
    +</center>
    +
    +**Note:** A table which is defined on a stream is internally not 
materialized. 
    +
    +### Continuous Queries
    +
    +A continuous query is evaluated on a dynamic table and produces a new 
dynamic table as result. In contrast to a batch query, a continuous query never 
terminates and updates its result table according to the updates on its input 
tables. At any point in time, the result of a continuous query is semantically 
equivalent to the result of the same query being executed in batch mode on a 
snapshot of the input tables. 
    +
    +In the following we show two example queries on a `clicks` table that is 
defined on the stream of click events.
    +
    +The first query is a simple `GROUP-BY COUNT` aggregation query. It groups 
the `clicks` table on the `user` field and counts the number of visited URLs. 
The following figure shows how the query is evaluated over time as the `clicks` 
table is updated with additional rows.
    +
    +<center>
    +<img alt="Continuous Non-Windowed Query" src="{{ site.baseurl 
}}/fig/table-streaming/query-groupBy-cnt.png" width="90%">
    --- End diff --
    
    This figure show the behavior of non-Windowed Query.  In common way, 
continuous query is Continuous query and non-windowed are perfect counterparts. 
But I think we can not assume that the data is a batch of updates. It should be 
update one by one in stream, especially processing-time mode. So I think the 
figure should be improved.(Update the result one by one). What do you think?
     
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to