You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/02/14 10:13:55 UTC

[GitHub] [flink] aljoscha opened a new pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

aljoscha opened a new pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092
 
 
   This is the second phase of https://cwiki.apache.org/confluence/display/FLINK/FLIP-42%3A+Rework+Flink+Documentation.
   
   The first few comments are preliminary refactorings of the doc that are also in separate PRs. You can also review them here if you want.
   
   The commits describe how the sections are moved around and when they are adapted/fixed. The general idea is that all concept-y material is extracted from individual sections and moved to the concepts section. On the way, some duplicate material was dropped and I fixed outdated references/naming.
   
   I also changed all links to use the `{{ site.baseurl }}{% link dev/bar/baz/doc.md %}` format which has detection of missing links built in. 
   
   I added `TODO: ...` entries in various places in the concepts sections to mark what still needs to be done. I think leaving them in for now is better than what we had before as a concepts section, I would create Jira Issues for those and try and spread the work. At the latest before the next release those should be removed.
   
   Reviewing this PR is probably easiest by checking it out and reading the concepts sections.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379555751
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
 
 Review comment:
   "Flink needs to be aware of the state in order to allow savepoints." does not make sense to me."

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379575236
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
 
 Review comment:
   all streaming programs, right?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149265863 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:6ecbdfc2ad86762e77113d6440d37800f86b0c17 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149280152 TriggerType:PUSH TriggerID:6ecbdfc2ad86762e77113d6440d37800f86b0c17
   Hash:6ecbdfc2ad86762e77113d6440d37800f86b0c17 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5247 TriggerType:PUSH TriggerID:6ecbdfc2ad86762e77113d6440d37800f86b0c17
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   * 419a65d99bfd92908136d010ad753f9b20cefb06 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149265863) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243) 
   * 6ecbdfc2ad86762e77113d6440d37800f86b0c17 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149280152) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5247) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379560406
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
 
 Review comment:
   As far as I remember, this was supposed to be about the conceptual differences between state handling in batch and stream processing. I don't remember exactly, what we talked about here. The only thing I can think of right now is, that the data structures in which state is stored need to be different in batch and stream processing for it to be efficient.
   
   I guess one could also say something about boundedness and expiry of state?
   
   @StephanEwen what did we have in mind here?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149265863 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   * 419a65d99bfd92908136d010ad753f9b20cefb06 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149265863) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586987015
 
 
   @aljoscha I see. How can we make sure, these comments are not lost. Directly create sub-tasks?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586988887
 
 
   @knaufk I think yes, on the subtasks.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379566803
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
 
 Review comment:
   ```suggestion
   support use cases where records of one stream need to be
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379569902
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
 
 Review comment:
   related is very unprecise

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379562658
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
 
 Review comment:
   I think, this introduction should already mention the concept and importance of state locality. Maybe with the typical figure of two-tiered architecture to state and logic fused into one thing.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379573424
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
 
 Review comment:
   style guide says use `<div class="alert alert-info"> // Info Message </div>` for notes

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379546166
 
 

 ##########
 File path: docs/concepts/stream-processing.md
 ##########
 @@ -0,0 +1,96 @@
+---
+title: Stream Processing
+nav-id: stream-processing
+nav-pos: 1
+nav-title: Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: Add introduction`
+* This will be replaced by the TOC
+{:toc}
+
+## A Unified System for Batch & Stream Processing
+
+`TODO`
+
+{% top %}
+
+## Programs and Dataflows
+
+The basic building blocks of Flink programs are **streams** and
+**transformations**. Conceptually a *stream* is a (potentially never-ending)
+flow of data records, and a *transformation* is an operation that takes one or
+more streams as input, and produces one or more output streams as a result.
+
+When executed, Flink programs are mapped to **streaming dataflows**, consisting
+of **streams** and transformation **operators**. Each dataflow starts with one
+or more **sources** and ends in one or more **sinks**. The dataflows resemble
+arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
+cycles are permitted via *iteration* constructs, for the most part we will
+gloss over this for simplicity.
+
+<img src="{{ site.baseurl }}/fig/program_dataflow.svg" alt="A DataStream program, and its dataflow." class="offset" width="80%" />
+
+Often there is a one-to-one correspondence between the transformations in the
+programs and the operators in the dataflow. Sometimes, however, one
 
 Review comment:
   ```suggestion
   programs and the operators in the logical dataflow graph. Sometimes, however, one
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379552116
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
 
 Review comment:
   Operations vs Operator vs Function vs Transformation?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379584150
 
 

 ##########
 File path: docs/concepts/flink-architecture.md
 ##########
 @@ -0,0 +1,140 @@
+---
+title: Flink Architecture
+nav-id: flink-architecture
+nav-pos: 4
+nav-title: Flink Architecture
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+* This will be replaced by the TOC
+{:toc}
+
+## Flink Applications and Flink Sessions
+
+`TODO: expand this section`
+
+{% top %}
+
+## Anatomy of a Flink Cluster
+
+`TODO: expand this section, especially about components of the Flink Master and
+container environments`
+
+The Flink runtime consists of two types of processes:
+
+  - The *Flink Master* coordinates the distributed execution. It schedules
+    tasks, coordinates checkpoints, coordinates recovery on failures, etc.
+
+    There is always at least one *Flink Master*. A high-availability setup will
+    have multiple *Flink Masters*, one of which one is always the *leader*, and
+    the others are *standby*.
+
+  - The *TaskManagers* (also called *workers*) execute the *tasks* (or more
+    specifically, the subtasks) of a dataflow, and buffer and exchange the data
+    *streams*.
+
+    There must always be at least one TaskManager.
+
+The Flink Master and TaskManagers can be started in various ways: directly on
+the machines as a [standalone cluster]({{ site.baseurl }}{% link
+ops/deployment/cluster_setup.md %}), in containers, or managed by resource
+frameworks like [YARN]({{ site.baseurl }}{% link ops/deployment/yarn_setup.md
+%}) or [Mesos]({{ site.baseurl }}{% link ops/deployment/mesos.md %}).
+TaskManagers connect to Flink Masters, announcing themselves as available, and
+are assigned work.
+
+The *client* is not part of the runtime and program execution, but is used to
+prepare and send a dataflow to the Flink Master.  After that, the client can
+disconnect, or stay connected to receive progress reports. The client runs
+either as part of the Java/Scala program that triggers the execution, or in the
+command line process `./bin/flink run ...`.
+
+<img src="{{ site.baseurl }}/fig/processes.svg" alt="The processes involved in executing a Flink dataflow" class="offset" width="80%" />
+
+{% top %}
+
+## Tasks and Operator Chains
+
+For distributed execution, Flink *chains* operator subtasks together into
+*tasks*. Each task is executed by one thread.  Chaining operators together into
+tasks is a useful optimization: it reduces the overhead of thread-to-thread
+handover and buffering, and increases overall throughput while decreasing
+latency.  The chaining behavior can be configured; see the [chaining docs]({{
+site.baseurl }}{% link dev/stream/operators/index.md
+%}#task-chaining-and-resource-groups) for details.
+
+The sample dataflow in the figure below is executed with five subtasks, and
+hence with five parallel threads.
+
+<img src="{{ site.baseurl }}/fig/tasks_chains.svg" alt="Operator chaining into Tasks" class="offset" width="80%" />
+
+{% top %}
+
+## Task Slots and Resources
+
+Each worker (TaskManager) is a *JVM process*, and may execute one or more
+subtasks in separate threads.  To control how many tasks a worker accepts, a
+worker has so called **task slots** (at least one).
+
+Each *task slot* represents a fixed subset of resources of the TaskManager. A
+TaskManager with three slots, for example, will dedicate 1/3 of its managed
+memory to each slot. Slotting the resources means that a subtask will not
+compete with subtasks from other jobs for managed memory, but instead has a
+certain amount of reserved managed memory. Note that no CPU isolation happens
+here; currently slots only separate the managed memory of tasks.
+
+By adjusting the number of task slots, users can define how subtasks are
+isolated from each other.  Having one slot per TaskManager means each task
+group runs in a separate JVM (which can be started in a separate container, for
+example). Having multiple slots means more subtasks share the same JVM. Tasks
+in the same JVM share TCP connections (via multiplexing) and heartbeat
+messages. They may also share data sets and data structures, thus reducing the
+per-task overhead.
+
+<img src="{{ site.baseurl }}/fig/tasks_slots.svg" alt="A TaskManager with Task Slots and Tasks" class="offset" width="80%" />
+
+By default, Flink allows subtasks to share slots even if they are subtasks of
+different tasks, so long as they are from the same job. The result is that one
 
 Review comment:
   ```suggestion
   different operators, so long as they are from the same job. The result is that one
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379554299
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
 
 Review comment:
   ```suggestion
   Flink needs to be aware of the state in order to it fault tolerant
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379550966
 
 

 ##########
 File path: docs/concepts/stream-processing.md
 ##########
 @@ -0,0 +1,96 @@
+---
+title: Stream Processing
+nav-id: stream-processing
+nav-pos: 1
+nav-title: Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: Add introduction`
+* This will be replaced by the TOC
+{:toc}
+
+## A Unified System for Batch & Stream Processing
+
+`TODO`
+
+{% top %}
+
+## Programs and Dataflows
+
+The basic building blocks of Flink programs are **streams** and
+**transformations**. Conceptually a *stream* is a (potentially never-ending)
+flow of data records, and a *transformation* is an operation that takes one or
+more streams as input, and produces one or more output streams as a result.
+
+When executed, Flink programs are mapped to **streaming dataflows**, consisting
+of **streams** and transformation **operators**. Each dataflow starts with one
+or more **sources** and ends in one or more **sinks**. The dataflows resemble
+arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
+cycles are permitted via *iteration* constructs, for the most part we will
+gloss over this for simplicity.
+
+<img src="{{ site.baseurl }}/fig/program_dataflow.svg" alt="A DataStream program, and its dataflow." class="offset" width="80%" />
+
+Often there is a one-to-one correspondence between the transformations in the
+programs and the operators in the dataflow. Sometimes, however, one
+transformation may consist of multiple transformation operators.
+
+{% top %}
+
+## Parallel Dataflows
+
+Programs in Flink are inherently parallel and distributed. During execution, a
+*stream* has one or more **stream partitions**, and each *operator* has one or
+more **operator subtasks**. The operator subtasks are independent of one
+another, and execute in different threads and possibly on different machines or
+containers.
+
+The number of operator subtasks is the **parallelism** of that particular
+operator. The parallelism of a stream is always that of its producing operator.
+Different operators of the same program may have different levels of
+parallelism.
+
+<img src="{{ site.baseurl }}/fig/parallel_dataflow.svg" alt="A parallel dataflow" class="offset" width="80%" />
+
+Streams can transport data between two operators in a *one-to-one* (or
 
 Review comment:
   To me the way Fabian presents theses data exchange patterns in his book is easier to understand. I think it was: 
   
   * Forward
   * Broadcast
   * Random
   * Keyed
   
   IMHO the additional classification in "Redistributing" and "One-to-one" does not help. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379581742
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
+**savepoint**. Savepoints allow both updating your programs and your Flink
+cluster without losing any state. 
+
+[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this. During execution programs are periodically snapshotted on
+the worker nodes and produce checkpoints. For recovery only the last completed
+checkpoint is needed and older checkpoints can be safely discarded as soon as a
+new one is completed.
+
+Savepoints are similar to these periodic checkpoints except that they are
+**triggered by the user** and **don't automatically expire** when newer
+checkpoints are completed. Savepoints can be created from the [command line]({{
+site.baseurl }}{% link ops/cli.md %}#savepoints) or when cancelling a job via
+the [REST API]({{ site.baseurl }}{% link monitoring/rest_api.md
+%}#cancel-job-with-savepoint).
+
+{% top %}
+
+### Exactly Once vs. At Least Once
+
+The alignment step may add latency to the streaming program. Usually, this
+extra latency is on the order of a few milliseconds, but we have seen cases
+where the latency of some outliers increased noticeably. For applications that
+require consistently super low latencies (few milliseconds) for all records,
+Flink has a switch to skip the stream alignment during a checkpoint. Checkpoint
+snapshots are still drawn as soon as an operator has seen the checkpoint
+barrier from each input.
+
+When the alignment is skipped, an operator keeps processing all inputs, even
+after some checkpoint barriers for checkpoint *n* arrived. That way, the
+operator also processes elements that belong to checkpoint *n+1* before the
+state snapshot for checkpoint *n* was taken.  On a restore, these records will
+occur as duplicates, because they are both included in the state snapshot of
+checkpoint *n*, and will be replayed as part of the data after checkpoint *n*.
+
+{% info Note %} Alignment happens only for operators with multiple predecessors
+(joins) as well as operators with multiple senders (after a stream
+repartitioning/shuffle).  Because of that, dataflows with only embarrassingly
+parallel streaming operations (`map()`, `flatMap()`, `filter()`, ...) actually
+give *exactly once* guarantees even in *at least once* mode.
+
+{% top %}
+
+## End-to-end Exactly-Once Programs
+
+`TODO: add`
+
+## State and Fault Tolerance in Batch Programs
+
+Flink executes [batch programs](../dev/batch/index.html) as a special case of
+streaming programs, where the streams are bounded (finite number of elements).
+A *DataSet* is treated internally as a stream of data. The concepts above thus
+apply to batch programs in the same way as well as they apply to streaming
+programs, with minor exceptions:
+
+  - [Fault tolerance for batch programs](../dev/batch/fault_tolerance.html)
+    does not use checkpointing.  Recovery happens by fully replaying the
+    streams.  That is possible, because inputs are bounded. This pushes the
+    cost more towards the recovery, but makes the regular processing cheaper,
+    because it avoids checkpoints.
+
+  - Stateful operations in the DataSet API use simplified in-memory/out-of-core
+    data structures, rather than key/value indexes.
+
+  - The DataSet API introduces special synchronized (superstep-based)
 
 Review comment:
   Are iterations about state?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379556110
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
 
 Review comment:
   Queryable State allows your to...

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379571869
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
 
 Review comment:
   ability twice

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] StephanEwen commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586403438
 
 
   Thanks, this is a great first step!
   
   Is the plan to merge this with the TODO comments? (I am personally fine with that)
   Or is that a base PR on which you want to base the next ones?
   
   A few minor suggestions:
     - I would remove the "Programming Model (outdated)" section.
     - Instead, having a staring page (maybe for concepts, maybe for the docs as a whole) that show the stack (SQL/Table API / DataStream API / StateFun, those on top of the common runtime)
   
     - I would drop operator state from the introduction. Instead I would mention it in the `DataStream API` as something that is mainly targeted towards connectors.
   
     - What do you think about removing ingestion time from the "timely" docs? We could change it to just a short mention under "event time", or mention it not at all.
   
     - I would also remove the "Asynchronous State Snapshots" section and simply have a comment somewhere that all persistence operations are asynchronous.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379561027
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
 
 Review comment:
   ```suggestion
   state is only possible on *keyed streams*, i.e. after a keyed partitioned data exchange, and is
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586194469
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e (Fri Feb 14 10:16:13 UTC 2020)
   
    ✅no warnings
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379565697
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
 
 Review comment:
   I would not consider KeyGroups a conceptual topics, but rather an operational or internal. I might miss something though.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379570828
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
 
 Review comment:
   ```suggestion
   tolerance during execution with the recovery time (the number of records that
   ```
   glossary

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379587424
 
 

 ##########
 File path: docs/concepts/timely-stream-processing.md
 ##########
 @@ -0,0 +1,237 @@
+---
+title: Timely Stream Processing
+nav-id: timely-stream-processing
+nav-pos: 3
+nav-title: Timely Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: add introduction`
+
+* This will be replaced by the TOC
+{:toc}
+
+## Latency & Completeness
+
+`TODO: add these two sections`
+
+### Latency vs. Completeness in Batch & Stream Processing
+
+{% top %}
+
+## Event Time, Processing Time, and Ingestion Time
+
+When referring to time in a streaming program (for example to define windows),
+one can refer to different notions of *time*:
+
+- **Processing time:** Processing time refers to the system time of the machine
+  that is executing the respective operation.
+
+  When a streaming program runs on processing time, all time-based operations
+  (like time windows) will use the system clock of the machines that run the
+  respective operator. An hourly processing time window will include all
+  records that arrived at a specific operator between the times when the system
+  clock indicated the full hour. For example, if an application begins running
+  at 9:15am, the first hourly processing time window will include events
+  processed between 9:15am and 10:00am, the next window will include events
+  processed between 10:00am and 11:00am, and so on.
+
+  Processing time is the simplest notion of time and requires no coordination
+  between streams and machines.  It provides the best performance and the
+  lowest latency. However, in distributed and asynchronous environments
+  processing time does not provide determinism, because it is susceptible to
+  the speed at which records arrive in the system (for example from the message
+  queue), to the speed at which the records flow between operators inside the
+  system, and to outages (scheduled, or otherwise).
+
+- **Event time:** Event time is the time that each individual event occurred on
+  its producing device.  This time is typically embedded within the records
+  before they enter Flink, and that *event timestamp* can be extracted from
+  each record. In event time, the progress of time depends on the data, not on
+  any wall clocks. Event time programs must specify how to generate *Event Time
+  Watermarks*, which is the mechanism that signals progress in event time. This
+  watermarking mechanism is described in a later section,
+  [below](#event-time-and-watermarks).
+
+  In a perfect world, event time processing would yield completely consistent
+  and deterministic results, regardless of when events arrive, or their
+  ordering.  However, unless the events are known to arrive in-order (by
+  timestamp), event time processing incurs some latency while waiting for
+  out-of-order events. As it is only possible to wait for a finite period of
+  time, this places a limit on how deterministic event time applications can
+  be.
+
+  Assuming all of the data has arrived, event time operations will behave as
+  expected, and produce correct and consistent results even when working with
+  out-of-order or late events, or when reprocessing historic data. For example,
+  an hourly event time window will contain all records that carry an event
+  timestamp that falls into that hour, regardless of the order in which they
+  arrive, or when they are processed. (See the section on [late
+  events](#late-elements) for more information.)
+
+
+
+  Note that sometimes when event time programs are processing live data in
+  real-time, they will use some *processing time* operations in order to
+  guarantee that they are progressing in a timely fashion.
+
+- **Ingestion time:** Ingestion time is the time that events enter Flink. At
+  the source operator each record gets the source's current time as a
+  timestamp, and time-based operations (like time windows) refer to that
+  timestamp.
+
+  *Ingestion time* sits conceptually in between *event time* and *processing
+  time*. Compared to *processing time*, it is slightly more expensive, but
+  gives more predictable results. Because *ingestion time* uses stable
+  timestamps (assigned once at the source), different window operations over
+  the records will refer to the same timestamp, whereas in *processing time*
+  each window operator may assign the record to a different window (based on
+  the local system clock and any transport delay).
+
+  Compared to *event time*, *ingestion time* programs cannot handle any
+  out-of-order events or late data, but the programs don't have to specify how
+  to generate *watermarks*.
+
+  Internally, *ingestion time* is treated much like *event time*, but with
+  automatic timestamp assignment and automatic watermark generation.
+
+<img src="{{ site.baseurl }}/fig/event_ingestion_processing_time.svg" alt="Event Time, Ingestion Time, and Processing Time" class="offset" width="80%" />
+
+{% top %}
+
+## Event Time and Watermarks
+
+*Note: Flink implements many techniques from the Dataflow Model. For a good
+introduction to event time and watermarks, have a look at the articles below.*
+
+  - [Streaming
+    101](https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101) by
+    Tyler Akidau
+  - The [Dataflow Model
+    paper](https://research.google.com/pubs/archive/43864.pdf)
+
+
+A stream processor that supports *event time* needs a way to measure the
+progress of event time.  For example, a window operator that builds hourly
+windows needs to be notified when event time has passed beyond the end of an
+hour, so that the operator can close the window in progress.
+
+*Event time* can progress independently of *processing time* (measured by wall
+clocks).  For example, in one program the current *event time* of an operator
+may trail slightly behind the *processing time* (accounting for a delay in
+receiving the events), while both proceed at the same speed.  On the other
+hand, another streaming program might progress through weeks of event time with
+only a few seconds of processing, by fast-forwarding through some historic data
+already buffered in a Kafka topic (or another message queue).
+
+------
+
+The mechanism in Flink to measure progress in event time is **watermarks**.
+Watermarks flow as part of the data stream and carry a timestamp *t*. A
+*Watermark(t)* declares that event time has reached time *t* in that stream,
+meaning that there should be no more elements from the stream with a timestamp
+*t' <= t* (i.e. events with timestamps older or equal to the watermark).
+
+The figure below shows a stream of events with (logical) timestamps, and
+watermarks flowing inline. In this example the events are in order (with
+respect to their timestamps), meaning that the watermarks are simply periodic
+markers in the stream.
+
+<img src="{{ site.baseurl }}/fig/stream_watermark_in_order.svg" alt="A data stream with events (in order) and watermarks" class="center" width="65%" />
+
+Watermarks are crucial for *out-of-order* streams, as illustrated below, where
+the events are not ordered by their timestamps.  In general a watermark is a
+declaration that by that point in the stream, all events up to a certain
+timestamp should have arrived.  Once a watermark reaches an operator, the
+operator can advance its internal *event time clock* to the value of the
+watermark.
+
+<img src="{{ site.baseurl }}/fig/stream_watermark_out_of_order.svg" alt="A data stream with events (out of order) and watermarks" class="center" width="65%" />
+
+Note that event time is inherited by a freshly created stream element (or
+elements) from either the event that produced them or from watermark that
+triggered creation of those elements.
+
+### Watermarks in Parallel Streams
+
+Watermarks are generated at, or directly after, source functions. Each parallel
+subtask of a source function usually generates its watermarks independently.
+These watermarks define the event time at that particular parallel source.
+
+As the watermarks flow through the streaming program, they advance the event
+time at the operators where they arrive. Whenever an operator advances its
+event time, it generates a new watermark downstream for its successor
+operators.
+
+Some operators consume multiple input streams; a union, for example, or
+operators following a *keyBy(...)* or *partition(...)* function.  Such an
+operator's current event time is the minimum of its input streams' event times.
+As its input streams update their event times, so does the operator.
+
+The figure below shows an example of events and watermarks flowing through
+parallel streams, and operators tracking event time.
+
+<img src="{{ site.baseurl }}/fig/parallel_streams_watermarks.svg" alt="Parallel data streams and operators with events and watermarks" class="center" width="80%" />
+
+Note that the Kafka source supports per-partition watermarking, which you can
+read more about [here]({{ site.baseurl }}{% link
+dev/event_timestamps_watermarks.md %}#timestamps-per-kafka-partition).
+
+
+## Lateness
+
+It is possible that certain elements will violate the watermark condition,
+meaning that even after the *Watermark(t)* has occurred, more elements with
+timestamp *t' <= t* will occur. In fact, in many real world setups, certain
+elements can be arbitrarily delayed, making it impossible to specify a time by
+which all elements of a certain event timestamp will have occurred.
+Furthermore, even if the lateness can be bounded, delaying the watermarks by
+too much is often not desirable, because it causes too much delay in the
+evaluation of event time windows.
+
+For this reason, streaming programs may explicitly expect some *late* elements.
+Late elements are elements that arrive after the system's event time clock (as
+signaled by the watermarks) has already passed the time of the late element's
+timestamp. See [Allowed Lateness]({{ site.baseurl }}{% link
+dev/stream/operators/windows.md %}#allowed-lateness) for more information on
+how to work with late elements in event time windows.
+
+## Windowing
+
+Aggregating events (e.g., counts, sums) works differently on streams than in
+batch processing.  For example, it is impossible to count all elements in a
+stream, because streams are in general infinite (unbounded). Instead,
+aggregates on streams (counts, sums, etc), are scoped by **windows**, such as
+*"count over the last 5 minutes"*, or *"sum of the last 100 elements"*.
+
+Windows can be *time driven* (example: every 30 seconds) or *data driven*
+(example: every 100 elements).  One typically distinguishes different types of
+windows, such as *tumbling windows* (no overlap), *sliding windows* (with
+overlap), and *session windows* (punctuated by a gap of inactivity).
+
+<img src="{{ site.baseurl }}/fig/windows.svg" alt="Time- and Count Windows" class="offset" width="80%" />
+
+More window examples can be found in this [blog
 
 Review comment:
   Please check out this blog post for additional examples of windows or take a look a window documentation of the DataStream API. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379558964
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
 
 Review comment:
   I would remove this section completely. It tries to cover too much to early. There is a section about Statebackends where this can be explained in more detail.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379572141
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
 
 Review comment:
   I usually use snapshots to subsume checkpoints and savepoints. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379587809
 
 

 ##########
 File path: docs/dev/stream/state/index.md
 ##########
 @@ -25,23 +25,10 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Stateful functions and operators store data across the processing of individual elements/events, making state a critical building block for
-any type of more elaborate operation.
-
-For example:
-
-  - When an application searches for certain event patterns, the state will store the sequence of events encountered so far.
-  - When aggregating events per minute/hour/day, the state holds the pending aggregates.
-  - When training a machine learning model over a stream of data points, the state holds the current version of the model parameters.
-  - When historic data needs to be managed, the state allows efficient access to events that occurred in the past.
-
-Flink needs to be aware of the state in order to make state fault tolerant using [checkpoints](checkpointing.html) and to allow [savepoints]({{ site.baseurl }}/ops/state/savepoints.html) of streaming applications.
-
-Knowledge about the state also allows for rescaling Flink applications, meaning that Flink takes care of redistributing state across parallel instances.
-
-The [queryable state](queryable_state.html) feature of Flink allows you to access state from outside of Flink during runtime.
-
-When working with state, it might also be useful to read about [Flink's state backends]({{ site.baseurl }}/ops/state/state_backends.html). Flink provides different state backends that specify how and where state is stored. State can be located on Java's heap or off-heap. Depending on your state backend, Flink can also *manage* the state for the application, meaning Flink deals with the memory management (possibly spilling to disk if necessary) to allow applications to hold very large state. State backends can be configured without changing your application logic.
+In this section you will learn about the stateful abstractions that Flink
 
 Review comment:
   are abstractions stateful?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379546260
 
 

 ##########
 File path: docs/concepts/stream-processing.md
 ##########
 @@ -0,0 +1,96 @@
+---
+title: Stream Processing
+nav-id: stream-processing
+nav-pos: 1
+nav-title: Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: Add introduction`
+* This will be replaced by the TOC
+{:toc}
+
+## A Unified System for Batch & Stream Processing
+
+`TODO`
+
+{% top %}
+
+## Programs and Dataflows
+
+The basic building blocks of Flink programs are **streams** and
+**transformations**. Conceptually a *stream* is a (potentially never-ending)
+flow of data records, and a *transformation* is an operation that takes one or
+more streams as input, and produces one or more output streams as a result.
+
+When executed, Flink programs are mapped to **streaming dataflows**, consisting
+of **streams** and transformation **operators**. Each dataflow starts with one
+or more **sources** and ends in one or more **sinks**. The dataflows resemble
+arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
+cycles are permitted via *iteration* constructs, for the most part we will
+gloss over this for simplicity.
+
+<img src="{{ site.baseurl }}/fig/program_dataflow.svg" alt="A DataStream program, and its dataflow." class="offset" width="80%" />
+
+Often there is a one-to-one correspondence between the transformations in the
+programs and the operators in the dataflow. Sometimes, however, one
+transformation may consist of multiple transformation operators.
 
 Review comment:
   ```suggestion
   transformation may consist of multiple operators.
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379576359
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
+**savepoint**. Savepoints allow both updating your programs and your Flink
+cluster without losing any state. 
+
+[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this. During execution programs are periodically snapshotted on
+the worker nodes and produce checkpoints. For recovery only the last completed
+checkpoint is needed and older checkpoints can be safely discarded as soon as a
+new one is completed.
+
+Savepoints are similar to these periodic checkpoints except that they are
+**triggered by the user** and **don't automatically expire** when newer
+checkpoints are completed. Savepoints can be created from the [command line]({{
 
 Review comment:
   How savepoint are created is not conceptual content, but operational.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379574851
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
 
 Review comment:
   I agree with Stephan that this quite implementation detaily. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379576013
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
+**savepoint**. Savepoints allow both updating your programs and your Flink
+cluster without losing any state. 
+
+[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this. During execution programs are periodically snapshotted on
 
 Review comment:
   Everything starting from "During" is not about savepoints and can be removed imho.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379550966
 
 

 ##########
 File path: docs/concepts/stream-processing.md
 ##########
 @@ -0,0 +1,96 @@
+---
+title: Stream Processing
+nav-id: stream-processing
+nav-pos: 1
+nav-title: Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: Add introduction`
+* This will be replaced by the TOC
+{:toc}
+
+## A Unified System for Batch & Stream Processing
+
+`TODO`
+
+{% top %}
+
+## Programs and Dataflows
+
+The basic building blocks of Flink programs are **streams** and
+**transformations**. Conceptually a *stream* is a (potentially never-ending)
+flow of data records, and a *transformation* is an operation that takes one or
+more streams as input, and produces one or more output streams as a result.
+
+When executed, Flink programs are mapped to **streaming dataflows**, consisting
+of **streams** and transformation **operators**. Each dataflow starts with one
+or more **sources** and ends in one or more **sinks**. The dataflows resemble
+arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
+cycles are permitted via *iteration* constructs, for the most part we will
+gloss over this for simplicity.
+
+<img src="{{ site.baseurl }}/fig/program_dataflow.svg" alt="A DataStream program, and its dataflow." class="offset" width="80%" />
+
+Often there is a one-to-one correspondence between the transformations in the
+programs and the operators in the dataflow. Sometimes, however, one
+transformation may consist of multiple transformation operators.
+
+{% top %}
+
+## Parallel Dataflows
+
+Programs in Flink are inherently parallel and distributed. During execution, a
+*stream* has one or more **stream partitions**, and each *operator* has one or
+more **operator subtasks**. The operator subtasks are independent of one
+another, and execute in different threads and possibly on different machines or
+containers.
+
+The number of operator subtasks is the **parallelism** of that particular
+operator. The parallelism of a stream is always that of its producing operator.
+Different operators of the same program may have different levels of
+parallelism.
+
+<img src="{{ site.baseurl }}/fig/parallel_dataflow.svg" alt="A parallel dataflow" class="offset" width="80%" />
+
+Streams can transport data between two operators in a *one-to-one* (or
 
 Review comment:
   I think,  different redistribution patterns that Fabian it in his book is more to the point. I think it was: 
   
   * Forward
   * Broadcast
   * Random
   * Keyed
   
   IMHO the additional classification in "Redistributing" and "One-to-one" does not help. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379553632
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
 
 Review comment:
   This paragraph feels repetitive. Maybe mention that while the output of a stateless function only depends on the input, the output of a stateful function depends on the input as well as its current state. As the state is a function full history of events, the output of a stateful function can depend on the the input as well as all previous inputs.
   
   Maybe this is too theoretical or mathematica. It is a slightly different view on it though.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380158677
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
 
 Review comment:
   yes 👌 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha closed pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha closed pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092
 
 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379569140
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
 
 Review comment:
   Style guide says "use 'you', never 'we'". 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379582452
 
 

 ##########
 File path: docs/concepts/flink-architecture.md
 ##########
 @@ -0,0 +1,140 @@
+---
+title: Flink Architecture
+nav-id: flink-architecture
+nav-pos: 4
+nav-title: Flink Architecture
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+* This will be replaced by the TOC
+{:toc}
+
+## Flink Applications and Flink Sessions
+
+`TODO: expand this section`
+
+{% top %}
+
+## Anatomy of a Flink Cluster
+
+`TODO: expand this section, especially about components of the Flink Master and
+container environments`
+
+The Flink runtime consists of two types of processes:
+
+  - The *Flink Master* coordinates the distributed execution. It schedules
+    tasks, coordinates checkpoints, coordinates recovery on failures, etc.
+
+    There is always at least one *Flink Master*. A high-availability setup will
+    have multiple *Flink Masters*, one of which one is always the *leader*, and
+    the others are *standby*.
+
+  - The *TaskManagers* (also called *workers*) execute the *tasks* (or more
+    specifically, the subtasks) of a dataflow, and buffer and exchange the data
 
 Review comment:
   according to the glossary "(or more specifically, the subtasks)" does not make sense.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379575432
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
 
 Review comment:
   hence all programs that use checkpointing

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380164391
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
 
 Review comment:
   I'll change it to `marks a specific point`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379551776
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
 
 Review comment:
   ```suggestion
   across multiple events (for example window operators). These operations are
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379584764
 
 

 ##########
 File path: docs/concepts/flink-architecture.md
 ##########
 @@ -0,0 +1,140 @@
+---
+title: Flink Architecture
+nav-id: flink-architecture
+nav-pos: 4
+nav-title: Flink Architecture
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+* This will be replaced by the TOC
+{:toc}
+
+## Flink Applications and Flink Sessions
+
+`TODO: expand this section`
+
+{% top %}
+
+## Anatomy of a Flink Cluster
+
+`TODO: expand this section, especially about components of the Flink Master and
+container environments`
+
+The Flink runtime consists of two types of processes:
+
+  - The *Flink Master* coordinates the distributed execution. It schedules
+    tasks, coordinates checkpoints, coordinates recovery on failures, etc.
+
+    There is always at least one *Flink Master*. A high-availability setup will
+    have multiple *Flink Masters*, one of which one is always the *leader*, and
+    the others are *standby*.
+
+  - The *TaskManagers* (also called *workers*) execute the *tasks* (or more
+    specifically, the subtasks) of a dataflow, and buffer and exchange the data
+    *streams*.
+
+    There must always be at least one TaskManager.
+
+The Flink Master and TaskManagers can be started in various ways: directly on
+the machines as a [standalone cluster]({{ site.baseurl }}{% link
+ops/deployment/cluster_setup.md %}), in containers, or managed by resource
+frameworks like [YARN]({{ site.baseurl }}{% link ops/deployment/yarn_setup.md
+%}) or [Mesos]({{ site.baseurl }}{% link ops/deployment/mesos.md %}).
+TaskManagers connect to Flink Masters, announcing themselves as available, and
+are assigned work.
+
+The *client* is not part of the runtime and program execution, but is used to
+prepare and send a dataflow to the Flink Master.  After that, the client can
+disconnect, or stay connected to receive progress reports. The client runs
+either as part of the Java/Scala program that triggers the execution, or in the
+command line process `./bin/flink run ...`.
+
+<img src="{{ site.baseurl }}/fig/processes.svg" alt="The processes involved in executing a Flink dataflow" class="offset" width="80%" />
+
+{% top %}
+
+## Tasks and Operator Chains
+
+For distributed execution, Flink *chains* operator subtasks together into
+*tasks*. Each task is executed by one thread.  Chaining operators together into
+tasks is a useful optimization: it reduces the overhead of thread-to-thread
+handover and buffering, and increases overall throughput while decreasing
+latency.  The chaining behavior can be configured; see the [chaining docs]({{
+site.baseurl }}{% link dev/stream/operators/index.md
+%}#task-chaining-and-resource-groups) for details.
+
+The sample dataflow in the figure below is executed with five subtasks, and
+hence with five parallel threads.
+
+<img src="{{ site.baseurl }}/fig/tasks_chains.svg" alt="Operator chaining into Tasks" class="offset" width="80%" />
+
+{% top %}
+
+## Task Slots and Resources
+
+Each worker (TaskManager) is a *JVM process*, and may execute one or more
+subtasks in separate threads.  To control how many tasks a worker accepts, a
+worker has so called **task slots** (at least one).
+
+Each *task slot* represents a fixed subset of resources of the TaskManager. A
+TaskManager with three slots, for example, will dedicate 1/3 of its managed
+memory to each slot. Slotting the resources means that a subtask will not
+compete with subtasks from other jobs for managed memory, but instead has a
+certain amount of reserved managed memory. Note that no CPU isolation happens
+here; currently slots only separate the managed memory of tasks.
+
+By adjusting the number of task slots, users can define how subtasks are
+isolated from each other.  Having one slot per TaskManager means each task
+group runs in a separate JVM (which can be started in a separate container, for
+example). Having multiple slots means more subtasks share the same JVM. Tasks
+in the same JVM share TCP connections (via multiplexing) and heartbeat
+messages. They may also share data sets and data structures, thus reducing the
+per-task overhead.
+
+<img src="{{ site.baseurl }}/fig/tasks_slots.svg" alt="A TaskManager with Task Slots and Tasks" class="offset" width="80%" />
+
+By default, Flink allows subtasks to share slots even if they are subtasks of
+different tasks, so long as they are from the same job. The result is that one
+slot may hold an entire pipeline of the job. Allowing this *slot sharing* has
+two main benefits:
+
+  - A Flink cluster needs exactly as many task slots as the highest parallelism
+    used in the job.  No need to calculate how many tasks (with varying
+    parallelism) a program contains in total.
+
+  - It is easier to get better resource utilization. Without slot sharing, the
+    non-intensive *source/map()* subtasks would block as many resources as the
+    resource intensive *window* subtasks.  With slot sharing, increasing the
+    base parallelism in our example from two to six yields full utilization of
+    the slotted resources, while making sure that the heavy subtasks are fairly
+    distributed among the TaskManagers.
+
+<img src="{{ site.baseurl }}/fig/slot_sharing.svg" alt="TaskManagers with shared Task Slots" class="offset" width="80%" />
+
+The APIs also include a *[resource group]({{ site.baseurl }}{% link
 
 Review comment:
   From here onwards, this is not conceptual anymore in my opinion.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380161116
 
 

 ##########
 File path: docs/dev/stream/state/index.md
 ##########
 @@ -25,23 +25,10 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Stateful functions and operators store data across the processing of individual elements/events, making state a critical building block for
-any type of more elaborate operation.
-
-For example:
-
-  - When an application searches for certain event patterns, the state will store the sequence of events encountered so far.
-  - When aggregating events per minute/hour/day, the state holds the pending aggregates.
-  - When training a machine learning model over a stream of data points, the state holds the current version of the model parameters.
-  - When historic data needs to be managed, the state allows efficient access to events that occurred in the past.
-
-Flink needs to be aware of the state in order to make state fault tolerant using [checkpoints](checkpointing.html) and to allow [savepoints]({{ site.baseurl }}/ops/state/savepoints.html) of streaming applications.
-
-Knowledge about the state also allows for rescaling Flink applications, meaning that Flink takes care of redistributing state across parallel instances.
-
-The [queryable state](queryable_state.html) feature of Flink allows you to access state from outside of Flink during runtime.
-
-When working with state, it might also be useful to read about [Flink's state backends]({{ site.baseurl }}/ops/state/state_backends.html). Flink provides different state backends that specify how and where state is stored. State can be located on Java's heap or off-heap. Depending on your state backend, Flink can also *manage* the state for the application, meaning Flink deals with the memory management (possibly spilling to disk if necessary) to allow applications to hold very large state. State backends can be configured without changing your application logic.
+In this section you will learn about the stateful abstractions that Flink
 
 Review comment:
   `learn about the APIs that Flink provides for writing stateful programs`?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379571150
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
 
 Review comment:
   usually in a distributed filesystem?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586980424
 
 
   I now also addressed @knaufk's comments. I did not address comments that "go deeper":
    - I didn't update figures, except some very simple changes
    - I didn't change how we refer to *dataflow*/*graph*/*logical graph* and the like because I think that requires changes in more parts
    - I didn't change how we refer to the distribution patterns of streams
    - I didn't change how we refer to *task*/*subtask*/*operator*/*operation*/*transformation*
   
   I don't think that we shouldn't do this at all but I don't have the capacity to do all of this in one go. IMO it's better to first get all the basic reorganisation in place and then to more passes over the doc to improve it continuously.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379583881
 
 

 ##########
 File path: docs/concepts/flink-architecture.md
 ##########
 @@ -0,0 +1,140 @@
+---
+title: Flink Architecture
+nav-id: flink-architecture
+nav-pos: 4
+nav-title: Flink Architecture
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+* This will be replaced by the TOC
+{:toc}
+
+## Flink Applications and Flink Sessions
+
+`TODO: expand this section`
+
+{% top %}
+
+## Anatomy of a Flink Cluster
+
+`TODO: expand this section, especially about components of the Flink Master and
+container environments`
+
+The Flink runtime consists of two types of processes:
+
+  - The *Flink Master* coordinates the distributed execution. It schedules
+    tasks, coordinates checkpoints, coordinates recovery on failures, etc.
+
+    There is always at least one *Flink Master*. A high-availability setup will
+    have multiple *Flink Masters*, one of which one is always the *leader*, and
+    the others are *standby*.
+
+  - The *TaskManagers* (also called *workers*) execute the *tasks* (or more
+    specifically, the subtasks) of a dataflow, and buffer and exchange the data
+    *streams*.
+
+    There must always be at least one TaskManager.
+
+The Flink Master and TaskManagers can be started in various ways: directly on
+the machines as a [standalone cluster]({{ site.baseurl }}{% link
+ops/deployment/cluster_setup.md %}), in containers, or managed by resource
+frameworks like [YARN]({{ site.baseurl }}{% link ops/deployment/yarn_setup.md
+%}) or [Mesos]({{ site.baseurl }}{% link ops/deployment/mesos.md %}).
+TaskManagers connect to Flink Masters, announcing themselves as available, and
+are assigned work.
+
+The *client* is not part of the runtime and program execution, but is used to
+prepare and send a dataflow to the Flink Master.  After that, the client can
+disconnect, or stay connected to receive progress reports. The client runs
+either as part of the Java/Scala program that triggers the execution, or in the
+command line process `./bin/flink run ...`.
+
+<img src="{{ site.baseurl }}/fig/processes.svg" alt="The processes involved in executing a Flink dataflow" class="offset" width="80%" />
+
+{% top %}
+
+## Tasks and Operator Chains
+
+For distributed execution, Flink *chains* operator subtasks together into
+*tasks*. Each task is executed by one thread.  Chaining operators together into
+tasks is a useful optimization: it reduces the overhead of thread-to-thread
+handover and buffering, and increases overall throughput while decreasing
+latency.  The chaining behavior can be configured; see the [chaining docs]({{
+site.baseurl }}{% link dev/stream/operators/index.md
+%}#task-chaining-and-resource-groups) for details.
+
+The sample dataflow in the figure below is executed with five subtasks, and
+hence with five parallel threads.
+
+<img src="{{ site.baseurl }}/fig/tasks_chains.svg" alt="Operator chaining into Tasks" class="offset" width="80%" />
+
+{% top %}
+
+## Task Slots and Resources
+
+Each worker (TaskManager) is a *JVM process*, and may execute one or more
+subtasks in separate threads.  To control how many tasks a worker accepts, a
+worker has so called **task slots** (at least one).
+
+Each *task slot* represents a fixed subset of resources of the TaskManager. A
 
 Review comment:
   There is already quite some operational content mixed in here. Could be shorter in the Concepts section in my opinion.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379567553
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
 
 Review comment:
   ```suggestion
   broadcasted to all downstream tasks, where they are used to maintain the same state among all subtasks. This state can then be accessed while processing records of a second stream.
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379547482
 
 

 ##########
 File path: docs/concepts/stream-processing.md
 ##########
 @@ -0,0 +1,96 @@
+---
+title: Stream Processing
+nav-id: stream-processing
+nav-pos: 1
+nav-title: Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: Add introduction`
+* This will be replaced by the TOC
+{:toc}
+
+## A Unified System for Batch & Stream Processing
+
+`TODO`
+
+{% top %}
+
+## Programs and Dataflows
+
+The basic building blocks of Flink programs are **streams** and
+**transformations**. Conceptually a *stream* is a (potentially never-ending)
+flow of data records, and a *transformation* is an operation that takes one or
+more streams as input, and produces one or more output streams as a result.
+
+When executed, Flink programs are mapped to **streaming dataflows**, consisting
+of **streams** and transformation **operators**. Each dataflow starts with one
+or more **sources** and ends in one or more **sinks**. The dataflows resemble
+arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
+cycles are permitted via *iteration* constructs, for the most part we will
+gloss over this for simplicity.
+
+<img src="{{ site.baseurl }}/fig/program_dataflow.svg" alt="A DataStream program, and its dataflow." class="offset" width="80%" />
+
+Often there is a one-to-one correspondence between the transformations in the
+programs and the operators in the dataflow. Sometimes, however, one
+transformation may consist of multiple transformation operators.
+
+{% top %}
+
+## Parallel Dataflows
+
+Programs in Flink are inherently parallel and distributed. During execution, a
+*stream* has one or more **stream partitions**, and each *operator* has one or
+more **operator subtasks**. The operator subtasks are independent of one
+another, and execute in different threads and possibly on different machines or
+containers.
+
+The number of operator subtasks is the **parallelism** of that particular
+operator. The parallelism of a stream is always that of its producing operator.
+Different operators of the same program may have different levels of
+parallelism.
+
+<img src="{{ site.baseurl }}/fig/parallel_dataflow.svg" alt="A parallel dataflow" class="offset" width="80%" />
 
 Review comment:
   I would update this figure. 
   
   Top: Logical (Data Flow) Graph 
   Bottom: Physical Graph
   
   In the bottom graph, all subtasks for an operator together are *not* an operator. It looks like it in the figure.
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149265863 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:6ecbdfc2ad86762e77113d6440d37800f86b0c17 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149280152 TriggerType:PUSH TriggerID:6ecbdfc2ad86762e77113d6440d37800f86b0c17
   Hash:6ecbdfc2ad86762e77113d6440d37800f86b0c17 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5247 TriggerType:PUSH TriggerID:6ecbdfc2ad86762e77113d6440d37800f86b0c17
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   * 419a65d99bfd92908136d010ad753f9b20cefb06 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149265863) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243) 
   * 6ecbdfc2ad86762e77113d6440d37800f86b0c17 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149280152) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5247) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379587916
 
 

 ##########
 File path: docs/dev/stream/state/state.md
 ##########
 @@ -22,66 +22,17 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-This document explains how to use Flink's state abstractions when developing an application.
+In this section you will learn about the stateful abstractions that Flink
 
 Review comment:
   see above

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586944421
 
 
   Thanks a lot for the reviews!
   
   Regarding TODOs about extending sections: I would prefer to leave them in, merge this PR rather quickly, and create follow-up Jira Issues. This reworking is already big enough as it is and if we try to to the one big everything-solution I'm afraid this will drag on quite a bit. Incremental improvements should work well here. I will therefore also not address all comments, for example about figures or longer sections that I would have to newly create or that would take too much time now. I think this change is already an improvement on the previous state, as is.
   
   @StephanEwen I addressed your comments. Some of them I already had as TODOs in the document, it's good to see them confirmed and I not did the actual changed.
   
   @knaufk I have yet to go through your suggestions and apply them.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380159704
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
+**savepoint**. Savepoints allow both updating your programs and your Flink
+cluster without losing any state. 
+
+[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this. During execution programs are periodically snapshotted on
+the worker nodes and produce checkpoints. For recovery only the last completed
+checkpoint is needed and older checkpoints can be safely discarded as soon as a
+new one is completed.
+
+Savepoints are similar to these periodic checkpoints except that they are
+**triggered by the user** and **don't automatically expire** when newer
+checkpoints are completed. Savepoints can be created from the [command line]({{
+site.baseurl }}{% link ops/cli.md %}#savepoints) or when cancelling a job via
+the [REST API]({{ site.baseurl }}{% link monitoring/rest_api.md
+%}#cancel-job-with-savepoint).
+
+{% top %}
+
+### Exactly Once vs. At Least Once
+
+The alignment step may add latency to the streaming program. Usually, this
+extra latency is on the order of a few milliseconds, but we have seen cases
+where the latency of some outliers increased noticeably. For applications that
+require consistently super low latencies (few milliseconds) for all records,
+Flink has a switch to skip the stream alignment during a checkpoint. Checkpoint
+snapshots are still drawn as soon as an operator has seen the checkpoint
+barrier from each input.
+
+When the alignment is skipped, an operator keeps processing all inputs, even
+after some checkpoint barriers for checkpoint *n* arrived. That way, the
+operator also processes elements that belong to checkpoint *n+1* before the
+state snapshot for checkpoint *n* was taken.  On a restore, these records will
+occur as duplicates, because they are both included in the state snapshot of
+checkpoint *n*, and will be replayed as part of the data after checkpoint *n*.
+
+{% info Note %} Alignment happens only for operators with multiple predecessors
+(joins) as well as operators with multiple senders (after a stream
+repartitioning/shuffle).  Because of that, dataflows with only embarrassingly
+parallel streaming operations (`map()`, `flatMap()`, `filter()`, ...) actually
+give *exactly once* guarantees even in *at least once* mode.
+
+{% top %}
+
+## End-to-end Exactly-Once Programs
+
+`TODO: add`
+
+## State and Fault Tolerance in Batch Programs
+
+Flink executes [batch programs](../dev/batch/index.html) as a special case of
+streaming programs, where the streams are bounded (finite number of elements).
+A *DataSet* is treated internally as a stream of data. The concepts above thus
+apply to batch programs in the same way as well as they apply to streaming
+programs, with minor exceptions:
+
+  - [Fault tolerance for batch programs](../dev/batch/fault_tolerance.html)
+    does not use checkpointing.  Recovery happens by fully replaying the
+    streams.  That is possible, because inputs are bounded. This pushes the
+    cost more towards the recovery, but makes the regular processing cheaper,
+    because it avoids checkpoints.
+
+  - Stateful operations in the DataSet API use simplified in-memory/out-of-core
+    data structures, rather than key/value indexes.
+
+  - The DataSet API introduces special synchronized (superstep-based)
 
 Review comment:
   in a way they are because iterations repeatedly join a stream with the existing state

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379566326
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
 
 Review comment:
   ```suggestion
   operator instances when the parallelism is changed. There are different
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379574304
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
 
 Review comment:
   Feels like Barriers, Snapshotting Operator State, Asynchronous State Snasphots and Recovery are sub-section of checkpointing.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379566028
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
 
 Review comment:
   parallel operator instance => sub-task?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379586538
 
 

 ##########
 File path: docs/concepts/timely-stream-processing.md
 ##########
 @@ -0,0 +1,237 @@
+---
+title: Timely Stream Processing
+nav-id: timely-stream-processing
+nav-pos: 3
+nav-title: Timely Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: add introduction`
+
+* This will be replaced by the TOC
+{:toc}
+
+## Latency & Completeness
+
+`TODO: add these two sections`
+
+### Latency vs. Completeness in Batch & Stream Processing
+
+{% top %}
+
+## Event Time, Processing Time, and Ingestion Time
+
+When referring to time in a streaming program (for example to define windows),
+one can refer to different notions of *time*:
+
+- **Processing time:** Processing time refers to the system time of the machine
+  that is executing the respective operation.
+
+  When a streaming program runs on processing time, all time-based operations
+  (like time windows) will use the system clock of the machines that run the
+  respective operator. An hourly processing time window will include all
+  records that arrived at a specific operator between the times when the system
+  clock indicated the full hour. For example, if an application begins running
+  at 9:15am, the first hourly processing time window will include events
+  processed between 9:15am and 10:00am, the next window will include events
+  processed between 10:00am and 11:00am, and so on.
+
+  Processing time is the simplest notion of time and requires no coordination
+  between streams and machines.  It provides the best performance and the
+  lowest latency. However, in distributed and asynchronous environments
+  processing time does not provide determinism, because it is susceptible to
+  the speed at which records arrive in the system (for example from the message
+  queue), to the speed at which the records flow between operators inside the
+  system, and to outages (scheduled, or otherwise).
+
+- **Event time:** Event time is the time that each individual event occurred on
+  its producing device.  This time is typically embedded within the records
+  before they enter Flink, and that *event timestamp* can be extracted from
+  each record. In event time, the progress of time depends on the data, not on
+  any wall clocks. Event time programs must specify how to generate *Event Time
+  Watermarks*, which is the mechanism that signals progress in event time. This
+  watermarking mechanism is described in a later section,
+  [below](#event-time-and-watermarks).
+
+  In a perfect world, event time processing would yield completely consistent
+  and deterministic results, regardless of when events arrive, or their
+  ordering.  However, unless the events are known to arrive in-order (by
+  timestamp), event time processing incurs some latency while waiting for
+  out-of-order events. As it is only possible to wait for a finite period of
+  time, this places a limit on how deterministic event time applications can
+  be.
+
+  Assuming all of the data has arrived, event time operations will behave as
+  expected, and produce correct and consistent results even when working with
+  out-of-order or late events, or when reprocessing historic data. For example,
+  an hourly event time window will contain all records that carry an event
+  timestamp that falls into that hour, regardless of the order in which they
+  arrive, or when they are processed. (See the section on [late
+  events](#late-elements) for more information.)
+
+
+
+  Note that sometimes when event time programs are processing live data in
+  real-time, they will use some *processing time* operations in order to
+  guarantee that they are progressing in a timely fashion.
+
+- **Ingestion time:** Ingestion time is the time that events enter Flink. At
+  the source operator each record gets the source's current time as a
+  timestamp, and time-based operations (like time windows) refer to that
+  timestamp.
+
+  *Ingestion time* sits conceptually in between *event time* and *processing
+  time*. Compared to *processing time*, it is slightly more expensive, but
+  gives more predictable results. Because *ingestion time* uses stable
+  timestamps (assigned once at the source), different window operations over
+  the records will refer to the same timestamp, whereas in *processing time*
+  each window operator may assign the record to a different window (based on
+  the local system clock and any transport delay).
+
+  Compared to *event time*, *ingestion time* programs cannot handle any
+  out-of-order events or late data, but the programs don't have to specify how
+  to generate *watermarks*.
+
+  Internally, *ingestion time* is treated much like *event time*, but with
+  automatic timestamp assignment and automatic watermark generation.
+
+<img src="{{ site.baseurl }}/fig/event_ingestion_processing_time.svg" alt="Event Time, Ingestion Time, and Processing Time" class="offset" width="80%" />
+
+{% top %}
+
+## Event Time and Watermarks
+
+*Note: Flink implements many techniques from the Dataflow Model. For a good
+introduction to event time and watermarks, have a look at the articles below.*
+
+  - [Streaming
+    101](https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101) by
+    Tyler Akidau
+  - The [Dataflow Model
+    paper](https://research.google.com/pubs/archive/43864.pdf)
+
+
+A stream processor that supports *event time* needs a way to measure the
+progress of event time.  For example, a window operator that builds hourly
+windows needs to be notified when event time has passed beyond the end of an
+hour, so that the operator can close the window in progress.
+
+*Event time* can progress independently of *processing time* (measured by wall
+clocks).  For example, in one program the current *event time* of an operator
+may trail slightly behind the *processing time* (accounting for a delay in
+receiving the events), while both proceed at the same speed.  On the other
+hand, another streaming program might progress through weeks of event time with
+only a few seconds of processing, by fast-forwarding through some historic data
+already buffered in a Kafka topic (or another message queue).
+
+------
+
+The mechanism in Flink to measure progress in event time is **watermarks**.
+Watermarks flow as part of the data stream and carry a timestamp *t*. A
+*Watermark(t)* declares that event time has reached time *t* in that stream,
+meaning that there should be no more elements from the stream with a timestamp
+*t' <= t* (i.e. events with timestamps older or equal to the watermark).
+
+The figure below shows a stream of events with (logical) timestamps, and
+watermarks flowing inline. In this example the events are in order (with
+respect to their timestamps), meaning that the watermarks are simply periodic
+markers in the stream.
+
+<img src="{{ site.baseurl }}/fig/stream_watermark_in_order.svg" alt="A data stream with events (in order) and watermarks" class="center" width="65%" />
+
+Watermarks are crucial for *out-of-order* streams, as illustrated below, where
+the events are not ordered by their timestamps.  In general a watermark is a
+declaration that by that point in the stream, all events up to a certain
+timestamp should have arrived.  Once a watermark reaches an operator, the
+operator can advance its internal *event time clock* to the value of the
+watermark.
+
+<img src="{{ site.baseurl }}/fig/stream_watermark_out_of_order.svg" alt="A data stream with events (out of order) and watermarks" class="center" width="65%" />
+
+Note that event time is inherited by a freshly created stream element (or
+elements) from either the event that produced them or from watermark that
+triggered creation of those elements.
+
+### Watermarks in Parallel Streams
+
+Watermarks are generated at, or directly after, source functions. Each parallel
+subtask of a source function usually generates its watermarks independently.
+These watermarks define the event time at that particular parallel source.
+
+As the watermarks flow through the streaming program, they advance the event
+time at the operators where they arrive. Whenever an operator advances its
+event time, it generates a new watermark downstream for its successor
+operators.
+
+Some operators consume multiple input streams; a union, for example, or
+operators following a *keyBy(...)* or *partition(...)* function.  Such an
+operator's current event time is the minimum of its input streams' event times.
+As its input streams update their event times, so does the operator.
+
+The figure below shows an example of events and watermarks flowing through
+parallel streams, and operators tracking event time.
+
+<img src="{{ site.baseurl }}/fig/parallel_streams_watermarks.svg" alt="Parallel data streams and operators with events and watermarks" class="center" width="80%" />
+
+Note that the Kafka source supports per-partition watermarking, which you can
 
 Review comment:
   too specific

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379568312
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
 
 Review comment:
   "As an example ... one can imagine" is a bit long-winded introduction into an example ;) 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379567637
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
 
 Review comment:
   ### 
   ```suggestion
   As an example where
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379571692
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
 
 Review comment:
   ```suggestion
   effected the previously checkpointed state.
   ``

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-589643739
 
 
   Thanks for the reviews! I created new subtasks for the `TODOs` and for things that were mentioned in the review but which I didn't address.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379554124
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
 
 Review comment:
   is the sequence of events the state or does the state store the sequence of events? I would say the former. Same for the examples below.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380157926
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
 
 Review comment:
   you're right, I'll change that. 👌 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380158898
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
+**savepoint**. Savepoints allow both updating your programs and your Flink
+cluster without losing any state. 
+
+[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this. During execution programs are periodically snapshotted on
 
 Review comment:
   👌 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149265863 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243 TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   Hash:6ecbdfc2ad86762e77113d6440d37800f86b0c17 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:6ecbdfc2ad86762e77113d6440d37800f86b0c17
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   * 419a65d99bfd92908136d010ad753f9b20cefb06 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149265863) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5243) 
   * 6ecbdfc2ad86762e77113d6440d37800f86b0c17 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:419a65d99bfd92908136d010ad753f9b20cefb06 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:419a65d99bfd92908136d010ad753f9b20cefb06
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   * 419a65d99bfd92908136d010ad753f9b20cefb06 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379576154
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
+site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
+to enable and configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this ability. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+dev/connectors/guarantees.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state.  These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+
+### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line.  A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_barriers.svg" alt="Checkpoint barriers in data streams" style="width:60%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+<i>S<sub>n</sub></i>) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position <i>S<sub>n</sub></i>
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before <i>S<sub>n</sub></i>, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/stream_aligning.svg" alt="Aligning data streams at operators with multiple inputs" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+  - As soon as the operator receives snapshot barrier *n* from an incoming
+    stream, it cannot process any further records from that stream until it has
+    received the barrier *n* from the other inputs as well. Otherwise, it would
+    mix records that belong to snapshot *n* and with records that belong to
+    snapshot *n+1*.
+  - Streams that report barrier *n* are temporarily set aside. Records that are
+    received from these streams are not processed, but put into an input
+    buffer.
+  - Once the last stream has received barrier *n*, the operator emits all
+    pending outgoing records, and then emits snapshot *n* barriers itself.
+  - After that, it resumes processing records from all input streams,
+    processing records from the input buffers before processing the records
+    from the streams.
+
+### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+  - For each parallel stream data source, the offset/position in the stream
+    when the snapshot was started
+  - For each operator, a pointer to the state that was stored as part of the
+    snapshot
+
+<div style="text-align: center">
+  <img src="{{ site.baseurl }}/fig/checkpointing.svg" alt="Illustration of the Checkpointing Mechanism" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+### Asynchronous State Snapshots
+
+Note that the above described mechanism implies that operators stop processing
+input records while they are storing a snapshot of their state in the *state
+backend*. This *synchronous* state snapshot introduces a delay every time a
+snapshot is taken.
+
+It is possible to let an operator continue processing while it stores its state
+snapshot, effectively letting the state snapshots happen *asynchronously* in
+the background. To do that, the operator must be able to produce a state object
+that should be stored in a way such that further modifications to the operator
+state do not affect that state object. For example, *copy-on-write* data
+structures, such as are used in RocksDB, have this behavior.
+
+After receiving the checkpoint barriers on its inputs, the operator starts the
+asynchronous snapshot copying of its state. It immediately emits the barrier to
+its outputs and continues with the regular stream processing. Once the
+background copy process has completed, it acknowledges the checkpoint to the
+checkpoint coordinator (the JobManager). The checkpoint is now only complete
+after all sinks have received the barriers and all stateful operators have
+acknowledged their completed backup (which may be after the barriers reach the
+sinks).
+
+See [State Backends]({{ site.baseurl }}{% link ops/state/state_backends.md %})
+for details on the state snapshots.
+
+### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position <i>S<sub>k</sub></i>. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset <i>S<sub>k</sub></i>.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({{ site.baseurl }}{% link
+ops/state/state_backends.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store.  In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint.
+
+<img src="{{ site.baseurl }}/fig/checkpoints.svg" alt="checkpoints and snapshots" class="offset" width="60%" />
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+Programs written in the Data Stream API can resume execution from a
+**savepoint**. Savepoints allow both updating your programs and your Flink
+cluster without losing any state. 
+
+[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this. During execution programs are periodically snapshotted on
+the worker nodes and produce checkpoints. For recovery only the last completed
+checkpoint is needed and older checkpoints can be safely discarded as soon as a
+new one is completed.
+
+Savepoints are similar to these periodic checkpoints except that they are
 
 Review comment:
   ```suggestion
   Savepoints are similar to checkpoints except that they are
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379583328
 
 

 ##########
 File path: docs/concepts/flink-architecture.md
 ##########
 @@ -0,0 +1,140 @@
+---
+title: Flink Architecture
+nav-id: flink-architecture
+nav-pos: 4
+nav-title: Flink Architecture
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+* This will be replaced by the TOC
+{:toc}
+
+## Flink Applications and Flink Sessions
+
+`TODO: expand this section`
+
+{% top %}
+
+## Anatomy of a Flink Cluster
+
+`TODO: expand this section, especially about components of the Flink Master and
+container environments`
+
+The Flink runtime consists of two types of processes:
+
+  - The *Flink Master* coordinates the distributed execution. It schedules
+    tasks, coordinates checkpoints, coordinates recovery on failures, etc.
+
+    There is always at least one *Flink Master*. A high-availability setup will
+    have multiple *Flink Masters*, one of which one is always the *leader*, and
+    the others are *standby*.
+
+  - The *TaskManagers* (also called *workers*) execute the *tasks* (or more
+    specifically, the subtasks) of a dataflow, and buffer and exchange the data
+    *streams*.
+
+    There must always be at least one TaskManager.
+
+The Flink Master and TaskManagers can be started in various ways: directly on
+the machines as a [standalone cluster]({{ site.baseurl }}{% link
+ops/deployment/cluster_setup.md %}), in containers, or managed by resource
+frameworks like [YARN]({{ site.baseurl }}{% link ops/deployment/yarn_setup.md
+%}) or [Mesos]({{ site.baseurl }}{% link ops/deployment/mesos.md %}).
+TaskManagers connect to Flink Masters, announcing themselves as available, and
+are assigned work.
+
+The *client* is not part of the runtime and program execution, but is used to
+prepare and send a dataflow to the Flink Master.  After that, the client can
+disconnect, or stay connected to receive progress reports. The client runs
+either as part of the Java/Scala program that triggers the execution, or in the
+command line process `./bin/flink run ...`.
+
+<img src="{{ site.baseurl }}/fig/processes.svg" alt="The processes involved in executing a Flink dataflow" class="offset" width="80%" />
+
+{% top %}
+
+## Tasks and Operator Chains
+
+For distributed execution, Flink *chains* operator subtasks together into
+*tasks*. Each task is executed by one thread.  Chaining operators together into
+tasks is a useful optimization: it reduces the overhead of thread-to-thread
+handover and buffering, and increases overall throughput while decreasing
+latency.  The chaining behavior can be configured; see the [chaining docs]({{
+site.baseurl }}{% link dev/stream/operators/index.md
+%}#task-chaining-and-resource-groups) for details.
+
+The sample dataflow in the figure below is executed with five subtasks, and
+hence with five parallel threads.
+
+<img src="{{ site.baseurl }}/fig/tasks_chains.svg" alt="Operator chaining into Tasks" class="offset" width="80%" />
 
 Review comment:
   Top: Logical Graph, there are no tasks in the logical graph
   Bottom: Physical Graph

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379545948
 
 

 ##########
 File path: docs/concepts/stream-processing.md
 ##########
 @@ -0,0 +1,96 @@
+---
+title: Stream Processing
+nav-id: stream-processing
+nav-pos: 1
+nav-title: Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+`TODO: Add introduction`
+* This will be replaced by the TOC
+{:toc}
+
+## A Unified System for Batch & Stream Processing
+
+`TODO`
+
+{% top %}
+
+## Programs and Dataflows
+
+The basic building blocks of Flink programs are **streams** and
+**transformations**. Conceptually a *stream* is a (potentially never-ending)
+flow of data records, and a *transformation* is an operation that takes one or
+more streams as input, and produces one or more output streams as a result.
+
+When executed, Flink programs are mapped to **streaming dataflows**, consisting
+of **streams** and transformation **operators**. Each dataflow starts with one
+or more **sources** and ends in one or more **sinks**. The dataflows resemble
+arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
+cycles are permitted via *iteration* constructs, for the most part we will
+gloss over this for simplicity.
+
+<img src="{{ site.baseurl }}/fig/program_dataflow.svg" alt="A DataStream program, and its dataflow." class="offset" width="80%" />
 
 Review comment:
   In the glossary we call the "streaming dataflow" "logical graph" or "jobgraph". Might want to add this here.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380157494
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of events that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance.  The state of the streaming applications is stored at a
+configurable place (such as the master node, or HDFS).
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow.  The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+been part of the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
 
 Review comment:
   Most of the documentation uses the `{% info/warn %}` so I will not change it here. We can do that throughout the documentation in the future.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379569436
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
 
 Review comment:
   I think, the third bullet point is too detailed.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379582165
 
 

 ##########
 File path: docs/concepts/flink-architecture.md
 ##########
 @@ -0,0 +1,140 @@
+---
+title: Flink Architecture
+nav-id: flink-architecture
+nav-pos: 4
+nav-title: Flink Architecture
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+* This will be replaced by the TOC
+{:toc}
+
+## Flink Applications and Flink Sessions
+
+`TODO: expand this section`
+
+{% top %}
+
+## Anatomy of a Flink Cluster
+
+`TODO: expand this section, especially about components of the Flink Master and
+container environments`
+
+The Flink runtime consists of two types of processes:
+
+  - The *Flink Master* coordinates the distributed execution. It schedules
+    tasks, coordinates checkpoints, coordinates recovery on failures, etc.
+
+    There is always at least one *Flink Master*. A high-availability setup will
 
 Review comment:
   ```suggestion
       There is always at least one *Flink Master*. A high-availability setup might
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
knaufk commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r379570309
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism.  During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+`TODO: potentially leave out Operator State and Broadcast State from concepts documentation`
+
+## Operator State
+
+*Operator State* (or *non-keyed state*) is state that is is bound to one
+parallel operator instance.  The [Kafka Connector]({{ site.baseurl }}{% link
+dev/connectors/kafka.md %}) is a good motivating example for the use of
+Operator State in Flink. Each parallel instance of the Kafka consumer maintains
+a map of topic partitions and offsets as its Operator State.
+
+The Operator State interfaces support redistributing state among parallel
+operator instances when the parallelism is changed. There can be different
+schemes for doing this redistribution.
+
+## Broadcast State
+
+*Broadcast State* is a special type of *Operator State*.  It was introduced to
+support use cases where some data coming from one stream is required to be
+broadcasted to all downstream tasks, where it is stored locally and is used to
+process all incoming elements on the other stream. As an example where
+broadcast state can emerge as a natural fit, one can imagine a low-throughput
+stream containing a set of rules which we want to evaluate against all elements
+coming from another stream. Having the above type of use cases in mind,
+broadcast state differs from the rest of operator states in that:
+ 1. it has a map format,
+ 2. it is only available to specific operators that have as inputs a
+    *broadcasted* stream and a *non-broadcasted* one, and
+ 3. such an operator can have *multiple broadcast states* with different names.
+
+{% top %}
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint is related to a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the events from the point of the checkpoint.
 
 Review comment:
   ```suggestion
   operators and replaying the records from the point of the checkpoint.
   ```
   glossary

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#issuecomment-586205307
 
 
   <!--
   Meta data
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   Hash:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148938105 TriggerType:PUSH TriggerID:c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e
   -->
   ## CI report:
   
   * c67d117a5d7d1ef8bc7e3c07b1611b554a3c821e Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148938105) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5174) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11092: [FLINK-15999] Extract “Concepts” material from API/Library sections and start proper concepts section
URL: https://github.com/apache/flink/pull/11092#discussion_r380153616
 
 

 ##########
 File path: docs/concepts/stateful-stream-processing.md
 ##########
 @@ -0,0 +1,412 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+<!--
+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.
+-->
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators).  These operations are
+called **stateful**.
+
+Stateful functions and operators store data across the processing of individual
+elements/events, making state a critical building block for any type of more
+elaborate operation.
+
+For example:
+
+  - When an application searches for certain event patterns, the state will
+    store the sequence of events encountered so far.
+  - When aggregating events per minute/hour/day, the state holds the pending
+    aggregates.
+  - When training a machine learning model over a stream of data points, the
+    state holds the current version of the model parameters.
+  - When historic data needs to be managed, the state allows efficient access
+    to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make state fault tolerant
+using [checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.md
+%}) and to allow [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md
+%}) of streaming applications.
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+The [queryable state]({{ site.baseurl }}{% link
+dev/stream/state/queryable_state.md %}) feature of Flink allows you to access
+state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+provides different state backends that specify how and where state is stored.
+State can be located on Java's heap or off-heap. Depending on your state
+backend, Flink can also *manage* the state for the application, meaning Flink
+deals with the memory management (possibly spilling to disk if necessary) to
+allow applications to hold very large state. State backends can be configured
+without changing your application logic.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+There are different types of state in Flink, the most-used type of state is
+*Keyed State*. For special cases you can use *Operator State* and *Broadcast
+State*. *Broadcast State* is a special type of *Operator State*.
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store.  The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, after a *keyBy()* function, and is
+restricted to the values associated with the current event's key. Aligning the
+keys of streams and state makes sure that all state updates are local
+operations, guaranteeing consistency without transaction overhead.  This
+alignment also allows Flink to redistribute the state and adjust the stream
+partitioning transparently.
+
+<img src="{{ site.baseurl }}/fig/state_partitioning.svg" alt="State and Partitioning" class="offset" width="50%" />
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
 
 Review comment:
   It can be important to understand the behaviour of Flink in certain situations, but I'm happy to remove this.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services