lostluck commented on code in PR #27632:
URL: https://github.com/apache/beam/pull/27632#discussion_r1289372099


##########
website/www/site/content/en/contribute/runner-guide.md:
##########
@@ -160,64 +161,51 @@ it from the main input, which is processed one element at 
a time. The SDK/user
 prepares a `PCollection` adequately, the runner materializes it, and then the
 runner feeds it to the `DoFn`.
 
-What you will need to implement is to inspect the materialization requested for
-the side input, and prepare it appropriately, and corresponding interactions
-when a `DoFn` reads the side inputs.
-
-The details and available support code vary by language.
-
-**Java**
-
-If you are using one of the above `DoFnRunner` classes, then the interface for
-letting them request side inputs is
-[`SideInputReader`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputReader.java).
-It is a simple mapping from side input and window to a value. The `DoFnRunner`
-will perform a mapping with the
-[`WindowMappingFn`](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowMappingFn.java)
-to request the appropriate window so you do not worry about invoking this UDF.
-When using the Fn API, it will be the SDK harness that maps windows as well.
-
-A simple, but not necessarily optimal approach to building a
-[`SideInputReader`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputReader.java)
-is to use a state backend. In our Java support code, this is called
-[`StateInternals`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/StateInternals.java)
-and you can build a
-[`SideInputHandler`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java)
-that will use your `StateInternals` to materialize a `PCollection` into the
-appropriate side input view and then yield the value when requested for a
-particular side input and window.
+Unlike main input data, which is *pushed* by the runner to the `ParDo` 
(generally
+via the FnApi Data channel), side input data is *pulled* by the `ParDo`
+from the runner (generally over the FnAPI State channel).
+
+A side input is accessed via a specific `access_pattern`.
+There are currently two access patterns enumerated in the
+`StandardSideInputTypes` proto: `beam:side_input:iterable:v1` which indicates
+the runner must return all values in a PCollection corresponding to a specific
+window and `beam:side_input:multimap:v1` which indicates the runner must return
+all values corresponding to a specific key and window.
+Being able to serve these access patterns efficiently may influence how a
+runner materializes this PCollection.
+
+SideInputs can be detected by looking at the `side_inputs` map in the
+`ParDoPayload` of `ParDo` transforms.
+The `ParDo` operation itself is responsible for invoking the
+`window_mapping_fn` (before invoking the runner) and `view_fn` (on the
+runner-returned values), so the runner need not concern itself with these
+fields.
 
 When a side input is needed but the side input has no data associated with it
 for a given window, elements in that window must be deferred until the side
-input has some data. The aforementioned
+input has some data or the watermark has advances sufficiently such that
+we can be sure there will be no data for that window. The
 
[`PushBackSideInputDoFnRunner`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java)
-is used to implement this.
-
-**Python**
-
-In Python, 
[`SideInputMap`](https://beam.apache.org/releases/pydoc/2.0.0/apache_beam.transforms.html#apache_beam.transforms.sideinputs.SideInputMap)
 maps
-windows to side input values. The `WindowMappingFn` manifests as a simple
-function. See
-[sideinputs.py](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/sideinputs.py).
+is an example of implementing this.
 
 #### State and Timers
 
 _Main design document: 
[https://s.apache.org/beam-state](https://s.apache.org/beam-state)_
 
 When a `ParDo` includes state and timers, its execution on your runner is 
usually
-very different. See the full details beyond those covered here.
+very different. In particular, the state must be persisted when the bundle
+completes and retrieved for future bundles. Timers that are set must also be
+injected into future bundles as the watermark advances sufficiently.
 
-State and timers are partitioned per key and window. You may need or want to
+State and timers are partitioned per key and window, that is, a `DoFn`
+processing a given key must have a consistent view of the state and timers
+across all elements that share this key. You may need or want to
 explicitly shuffle data to support this.
+Once the watermark has passed the end of the window (plus an allowance for
+allowed lateness, if any), state associated with this window can be dropped.
 
-**Java**
-
-We provide
-[`StatefulDoFnRunner`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java)
-to help with state cleanup. The non-user-facing interface
-[`StateInternals`](https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/StateInternals.java)
-is what a runner generally implements, and then the Beam support code can use
-this to implement user-facing state.
+State setting and retrieval is performed on the FnAPI State channel, whereas
+timer setting and firing happens on the FnAPI Data channel.
 
 #### Splittable DoFn

Review Comment:
   Yup. I was looking at the wrong commit. Thanks!



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to