You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2022/06/22 15:19:11 UTC

[GitHub] [tvm-rfcs] Lunderberg commented on a diff in pull request #80: Asynchronous stage in software pipeline

Lunderberg commented on code in PR #80:
URL: https://github.com/apache/tvm-rfcs/pull/80#discussion_r903828513


##########
rfcs/0077-async-pipeline.md:
##########
@@ -0,0 +1,528 @@
+- Feature Name: Asynchronous stage in software pipeline
+- Authors: [Masahiro Masuda](https://github.com/masahi), [Wuwei Lin](https://github.com/vinx13/)
+- Start Date: (2022-06-17)
+
+# Summary
+This RFC proposes two TIR intrinsics and an additional annotation to the TIR software pipeline transform, to express asynchrony **within the device code**.
+Asynchrony is prevalent on the host (runtime) side, and this proposal is the first step toward bringing the notion of an asynchronous operation in the
+generated code.
+
+The most important component we should agree on is the model of synchronization: Coming up with a design that is general enough to be useful for diverse backends, while making sure that the chosen design can be translated to a low-level synchronization model of a particular backend, is highly non-trivial.
+The approach described in this document is motivated by a use case for NVIDIA GPUs, but we took some cares so that the design can be adopted by other backends. For example, if a backend has an asynchronous DMA engine, vector and tensor unit, we can specify that each of them runs asynchronously in different stages in a pipeline, with necessary synchronization between them.
+
+The proposed model may have diverged from conventional ones, but we believe that this is a good fit for the TIR software pipeline specifically.
+
+# Asynchronous stage in a software pipeline
+
+### Background: What is a software pipeline, and what does the TIR software pipeline transform do?
+
+Software pipeline is an optimization technique to improve instruction-level parallelism of a loop. For example, given this program:
+
+```python
+B = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+```
+
+the goal is to overlap the execution of two statements in the loop body, by letting the two statements operate on different iterations of the loop. This way, the second statement would no longer depend on the completion of the first statement in the same iteration.
+
+The TIR software pipeline transform enables such transformation at the TIR level. We annotate the loop in the above program to specify, for each statement in the loop, the “stage” and the “order” in the pipeline:
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+sch.annotate(i, "software_pipeline_order", [0, 1])
+```
+
+Given the annotation above, the TIR software pipeline transform would break up the loop into three parts: prologue, pipeline body and epilogue. Different “stage” in the pipeline body become independent of each other, and the integer value of “stage” tells how many iterations each statement goes ahead of its consumer.
+
+```python
+B = alloc([2])
+
+# Prologue
+B[0] = A[0]
+
+# Body
+for i in range(15):
+    B[(i + 1) % 2] = A[i] + 1
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+C[15] = B[1] + 1
+```
+
+The two statements in the body can potentially run in parallel, if the underlying HW supports out-of-order execution.
+
+### Making parallelism more explicit: Asynchronous pipeline
+
+What’s currently available today is, after all, a “software” pipeline: whether or not independent statements actually run in parallel is up to the underlying HW, and programmers have little control over it. Moreover, for in-order processors like Hexagon DSP, this transformation alone would not help.
+
+The goal of this work is to support HW-backed asynchrony in the pipeline. Asynchronous data movement is becoming increasingly important in GPU computing, and NPUs typically have multiple kinds of asynchronous units (DMA copy, vector & matrix compute etc). To exploit such hardware features, it’s essential that we express all kinds of available asynchronies in the IR.
+
+A user of the TIR software pipeline transform will be able to specify which data movement or compute block should become asynchronous by an additional annotation. For example, given the  annotation specifying that the first block in the pipeline be made async,
+
+```python
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+
+...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+...
+
+# "0" refers to the first element in te list [0, 1] above, i.e. the first block
+sch.annotate(i, "software_pipeline_async_stages", [0])
+```
+
+we generate the following IR. An asynchronous block is decorated with the `async_scope` attribute, and two intrinsics are inserted to express synchronization.
+
+```python
+B = alloc([2])
+
+# Prologue
+async_scope:
+   B[0] = A[0]
+   async_commit_stage(0)
+
+# Body
+for i in range(15):
+    async_scope:
+        B[(i + 1) % 2] = A[i] + 1
+    async_commit_stage(0)
+
+    async_wait_stage(0, 1)
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+async_wait_stage(0, 0)
+C[15] = B[1] + 1
+```
+
+**Semantics of the proposed intrinsics**. “stage” refers to the same notion in the TIR software pipeline.
+- `async_commit_stage(i)` : Group one or more invocation of async operations, and “commit” them to the `i`-th stage. The exact interpretation of “committing” can be up to each backend, but informally it signifies that a group of async operations are now in-flight. The group of operations committed together is awaited as one chunk, and thus they constitute the granularity at which the synchronization intrinsic discussed next operates on.

Review Comment:
   Would it be cleaner to express `async_commit_stage` as an annotation rather than a callable intrinsic?  I'm thinking something like the following:
   
   ```python
   # A stage includes one or more async_scope blocks, and defines a
   # group.  async_scope blocks may only occur directly within an
   # async_stage block, or as elements of a SeqStmt that occurs directly
   # within an async_stage block.  The group is committed at the site
   # where it is defined.
   with async_commit_stage(0):
       with async_scope:
           B[(i + 1) % 2] = A[i] + 1
   ```
   
   This way, there's less backtracking needed for a reader to determine which scopes are being launched, and a runtime wouldn't need to maintain state describing the stages to be launched next time it encounters a call to `async_commit_stage`.  This would also prevent cases where the scopes have been defined outside a conditional, but the `async_commit_stage` call exists inside a conditional.



##########
rfcs/0077-async-pipeline.md:
##########
@@ -0,0 +1,528 @@
+- Feature Name: Asynchronous stage in software pipeline
+- Authors: [Masahiro Masuda](https://github.com/masahi), [Wuwei Lin](https://github.com/vinx13/)
+- Start Date: (2022-06-17)
+
+# Summary
+This RFC proposes two TIR intrinsics and an additional annotation to the TIR software pipeline transform, to express asynchrony **within the device code**.
+Asynchrony is prevalent on the host (runtime) side, and this proposal is the first step toward bringing the notion of an asynchronous operation in the
+generated code.
+
+The most important component we should agree on is the model of synchronization: Coming up with a design that is general enough to be useful for diverse backends, while making sure that the chosen design can be translated to a low-level synchronization model of a particular backend, is highly non-trivial.
+The approach described in this document is motivated by a use case for NVIDIA GPUs, but we took some cares so that the design can be adopted by other backends. For example, if a backend has an asynchronous DMA engine, vector and tensor unit, we can specify that each of them runs asynchronously in different stages in a pipeline, with necessary synchronization between them.
+
+The proposed model may have diverged from conventional ones, but we believe that this is a good fit for the TIR software pipeline specifically.
+
+# Asynchronous stage in a software pipeline
+
+### Background: What is a software pipeline, and what does the TIR software pipeline transform do?
+
+Software pipeline is an optimization technique to improve instruction-level parallelism of a loop. For example, given this program:
+
+```python
+B = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+```
+
+the goal is to overlap the execution of two statements in the loop body, by letting the two statements operate on different iterations of the loop. This way, the second statement would no longer depend on the completion of the first statement in the same iteration.
+
+The TIR software pipeline transform enables such transformation at the TIR level. We annotate the loop in the above program to specify, for each statement in the loop, the “stage” and the “order” in the pipeline:
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+sch.annotate(i, "software_pipeline_order", [0, 1])
+```
+
+Given the annotation above, the TIR software pipeline transform would break up the loop into three parts: prologue, pipeline body and epilogue. Different “stage” in the pipeline body become independent of each other, and the integer value of “stage” tells how many iterations each statement goes ahead of its consumer.
+
+```python
+B = alloc([2])
+
+# Prologue
+B[0] = A[0]
+
+# Body
+for i in range(15):
+    B[(i + 1) % 2] = A[i] + 1
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+C[15] = B[1] + 1
+```
+
+The two statements in the body can potentially run in parallel, if the underlying HW supports out-of-order execution.
+
+### Making parallelism more explicit: Asynchronous pipeline
+
+What’s currently available today is, after all, a “software” pipeline: whether or not independent statements actually run in parallel is up to the underlying HW, and programmers have little control over it. Moreover, for in-order processors like Hexagon DSP, this transformation alone would not help.
+
+The goal of this work is to support HW-backed asynchrony in the pipeline. Asynchronous data movement is becoming increasingly important in GPU computing, and NPUs typically have multiple kinds of asynchronous units (DMA copy, vector & matrix compute etc). To exploit such hardware features, it’s essential that we express all kinds of available asynchronies in the IR.
+
+A user of the TIR software pipeline transform will be able to specify which data movement or compute block should become asynchronous by an additional annotation. For example, given the  annotation specifying that the first block in the pipeline be made async,
+
+```python
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+
+...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+...
+
+# "0" refers to the first element in te list [0, 1] above, i.e. the first block
+sch.annotate(i, "software_pipeline_async_stages", [0])
+```
+
+we generate the following IR. An asynchronous block is decorated with the `async_scope` attribute, and two intrinsics are inserted to express synchronization.
+
+```python
+B = alloc([2])
+
+# Prologue
+async_scope:
+   B[0] = A[0]
+   async_commit_stage(0)
+
+# Body
+for i in range(15):
+    async_scope:
+        B[(i + 1) % 2] = A[i] + 1
+    async_commit_stage(0)
+
+    async_wait_stage(0, 1)
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+async_wait_stage(0, 0)
+C[15] = B[1] + 1
+```
+
+**Semantics of the proposed intrinsics**. “stage” refers to the same notion in the TIR software pipeline.
+- `async_commit_stage(i)` : Group one or more invocation of async operations, and “commit” them to the `i`-th stage. The exact interpretation of “committing” can be up to each backend, but informally it signifies that a group of async operations are now in-flight. The group of operations committed together is awaited as one chunk, and thus they constitute the granularity at which the synchronization intrinsic discussed next operates on.

Review Comment:
   Ooh, this would also give a really clean notation for stages that consist of only a single scope.
   
   ```python
   # If only one async_scope exists, the annotation can be dropped.
   with async_stage(0):
       B[(i + 1) % 2] = A[i] + 1
   ```



##########
rfcs/0077-async-pipeline.md:
##########
@@ -0,0 +1,528 @@
+- Feature Name: Asynchronous stage in software pipeline
+- Authors: [Masahiro Masuda](https://github.com/masahi), [Wuwei Lin](https://github.com/vinx13/)
+- Start Date: (2022-06-17)
+
+# Summary
+This RFC proposes two TIR intrinsics and an additional annotation to the TIR software pipeline transform, to express asynchrony **within the device code**.
+Asynchrony is prevalent on the host (runtime) side, and this proposal is the first step toward bringing the notion of an asynchronous operation in the
+generated code.
+
+The most important component we should agree on is the model of synchronization: Coming up with a design that is general enough to be useful for diverse backends, while making sure that the chosen design can be translated to a low-level synchronization model of a particular backend, is highly non-trivial.
+The approach described in this document is motivated by a use case for NVIDIA GPUs, but we took some cares so that the design can be adopted by other backends. For example, if a backend has an asynchronous DMA engine, vector and tensor unit, we can specify that each of them runs asynchronously in different stages in a pipeline, with necessary synchronization between them.
+
+The proposed model may have diverged from conventional ones, but we believe that this is a good fit for the TIR software pipeline specifically.
+
+# Asynchronous stage in a software pipeline
+
+### Background: What is a software pipeline, and what does the TIR software pipeline transform do?
+
+Software pipeline is an optimization technique to improve instruction-level parallelism of a loop. For example, given this program:
+
+```python
+B = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+```
+
+the goal is to overlap the execution of two statements in the loop body, by letting the two statements operate on different iterations of the loop. This way, the second statement would no longer depend on the completion of the first statement in the same iteration.
+
+The TIR software pipeline transform enables such transformation at the TIR level. We annotate the loop in the above program to specify, for each statement in the loop, the “stage” and the “order” in the pipeline:
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+sch.annotate(i, "software_pipeline_order", [0, 1])
+```
+
+Given the annotation above, the TIR software pipeline transform would break up the loop into three parts: prologue, pipeline body and epilogue. Different “stage” in the pipeline body become independent of each other, and the integer value of “stage” tells how many iterations each statement goes ahead of its consumer.
+
+```python
+B = alloc([2])
+
+# Prologue
+B[0] = A[0]
+
+# Body
+for i in range(15):
+    B[(i + 1) % 2] = A[i] + 1
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+C[15] = B[1] + 1
+```
+
+The two statements in the body can potentially run in parallel, if the underlying HW supports out-of-order execution.
+
+### Making parallelism more explicit: Asynchronous pipeline
+
+What’s currently available today is, after all, a “software” pipeline: whether or not independent statements actually run in parallel is up to the underlying HW, and programmers have little control over it. Moreover, for in-order processors like Hexagon DSP, this transformation alone would not help.
+
+The goal of this work is to support HW-backed asynchrony in the pipeline. Asynchronous data movement is becoming increasingly important in GPU computing, and NPUs typically have multiple kinds of asynchronous units (DMA copy, vector & matrix compute etc). To exploit such hardware features, it’s essential that we express all kinds of available asynchronies in the IR.
+
+A user of the TIR software pipeline transform will be able to specify which data movement or compute block should become asynchronous by an additional annotation. For example, given the  annotation specifying that the first block in the pipeline be made async,
+
+```python
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+
+...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+...
+
+# "0" refers to the first element in te list [0, 1] above, i.e. the first block
+sch.annotate(i, "software_pipeline_async_stages", [0])
+```
+
+we generate the following IR. An asynchronous block is decorated with the `async_scope` attribute, and two intrinsics are inserted to express synchronization.
+
+```python
+B = alloc([2])
+
+# Prologue
+async_scope:
+   B[0] = A[0]
+   async_commit_stage(0)
+
+# Body
+for i in range(15):
+    async_scope:
+        B[(i + 1) % 2] = A[i] + 1
+    async_commit_stage(0)
+
+    async_wait_stage(0, 1)
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+async_wait_stage(0, 0)
+C[15] = B[1] + 1
+```
+
+**Semantics of the proposed intrinsics**. “stage” refers to the same notion in the TIR software pipeline.

Review Comment:
   I think this is the case, but want to verify: The "stage" should be used to represent logical data dependencies, and should not be used to represent a hardware resource, correct?



##########
rfcs/0077-async-pipeline.md:
##########
@@ -0,0 +1,528 @@
+- Feature Name: Asynchronous stage in software pipeline
+- Authors: [Masahiro Masuda](https://github.com/masahi), [Wuwei Lin](https://github.com/vinx13/)
+- Start Date: (2022-06-17)
+
+# Summary
+This RFC proposes two TIR intrinsics and an additional annotation to the TIR software pipeline transform, to express asynchrony **within the device code**.
+Asynchrony is prevalent on the host (runtime) side, and this proposal is the first step toward bringing the notion of an asynchronous operation in the
+generated code.
+
+The most important component we should agree on is the model of synchronization: Coming up with a design that is general enough to be useful for diverse backends, while making sure that the chosen design can be translated to a low-level synchronization model of a particular backend, is highly non-trivial.
+The approach described in this document is motivated by a use case for NVIDIA GPUs, but we took some cares so that the design can be adopted by other backends. For example, if a backend has an asynchronous DMA engine, vector and tensor unit, we can specify that each of them runs asynchronously in different stages in a pipeline, with necessary synchronization between them.
+
+The proposed model may have diverged from conventional ones, but we believe that this is a good fit for the TIR software pipeline specifically.
+
+# Asynchronous stage in a software pipeline
+
+### Background: What is a software pipeline, and what does the TIR software pipeline transform do?
+
+Software pipeline is an optimization technique to improve instruction-level parallelism of a loop. For example, given this program:
+
+```python
+B = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+```
+
+the goal is to overlap the execution of two statements in the loop body, by letting the two statements operate on different iterations of the loop. This way, the second statement would no longer depend on the completion of the first statement in the same iteration.
+
+The TIR software pipeline transform enables such transformation at the TIR level. We annotate the loop in the above program to specify, for each statement in the loop, the “stage” and the “order” in the pipeline:
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+sch.annotate(i, "software_pipeline_order", [0, 1])
+```
+
+Given the annotation above, the TIR software pipeline transform would break up the loop into three parts: prologue, pipeline body and epilogue. Different “stage” in the pipeline body become independent of each other, and the integer value of “stage” tells how many iterations each statement goes ahead of its consumer.
+
+```python
+B = alloc([2])
+
+# Prologue
+B[0] = A[0]
+
+# Body
+for i in range(15):
+    B[(i + 1) % 2] = A[i] + 1
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+C[15] = B[1] + 1
+```
+
+The two statements in the body can potentially run in parallel, if the underlying HW supports out-of-order execution.
+
+### Making parallelism more explicit: Asynchronous pipeline
+
+What’s currently available today is, after all, a “software” pipeline: whether or not independent statements actually run in parallel is up to the underlying HW, and programmers have little control over it. Moreover, for in-order processors like Hexagon DSP, this transformation alone would not help.
+
+The goal of this work is to support HW-backed asynchrony in the pipeline. Asynchronous data movement is becoming increasingly important in GPU computing, and NPUs typically have multiple kinds of asynchronous units (DMA copy, vector & matrix compute etc). To exploit such hardware features, it’s essential that we express all kinds of available asynchronies in the IR.
+
+A user of the TIR software pipeline transform will be able to specify which data movement or compute block should become asynchronous by an additional annotation. For example, given the  annotation specifying that the first block in the pipeline be made async,
+
+```python
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+
+...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+...
+
+# "0" refers to the first element in te list [0, 1] above, i.e. the first block
+sch.annotate(i, "software_pipeline_async_stages", [0])
+```
+
+we generate the following IR. An asynchronous block is decorated with the `async_scope` attribute, and two intrinsics are inserted to express synchronization.
+
+```python
+B = alloc([2])
+
+# Prologue
+async_scope:
+   B[0] = A[0]
+   async_commit_stage(0)
+
+# Body
+for i in range(15):
+    async_scope:
+        B[(i + 1) % 2] = A[i] + 1
+    async_commit_stage(0)
+
+    async_wait_stage(0, 1)
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+async_wait_stage(0, 0)
+C[15] = B[1] + 1
+```
+
+**Semantics of the proposed intrinsics**. “stage” refers to the same notion in the TIR software pipeline.
+- `async_commit_stage(i)` : Group one or more invocation of async operations, and “commit” them to the `i`-th stage. The exact interpretation of “committing” can be up to each backend, but informally it signifies that a group of async operations are now in-flight. The group of operations committed together is awaited as one chunk, and thus they constitute the granularity at which the synchronization intrinsic discussed next operates on.
+- `async_wait_stage(i, N)` : Block until only `N` **most recent** committed groups are still in-flight at the stage `i` . In other words, if there are `M` committed groups in-flight at the stage `i`, at the invocation of `async_wait_stage(i, N)`, `M - N` oldest committed groups would be forced to complete. `N` doesn’t have to be a constant, but some backends may require a constant count (e.g. PTX)
+
+They directly correspond to the async data movement instructions in CUDA (PTX): [`cp.async.commit_group`](https://docs.nvidia.com/cuda/parallel-thread-execution/index.html#data-movement-and-conversion-instructions-cp-async-commit-group) and [`cp.async.wait_group`](https://docs.nvidia.com/cuda/parallel-thread-execution/index.html#data-movement-and-conversion-instructions-cp-async-wait-group).
+
+The CUDA counterparts do not have the notion of “stage”, since there is only one kind of async operation (copy from global to shared memory) supported by the current generation of NVIDIA GPU (Ampere, at the time of writing). To support more general cases where there could be multiple kinds of async “engine”, each of which corresponds to a different stage in an async pipeline, TIR `async_commit_stage` and `async_wait_stage` take a “stage” parameter.
+
+**The role of async_scope**. `async_scope` is represented by `AttrStmt` with key `tir::attr::async_scope`. It is inserted to let later transform passes know that the enclosed statement is intended to run asynchronously. This way, the actual lowering to target-dependent asynchronous instructions
+can happen much later in the compilation flow, rather than before the software pipeline transform using tensorization. For example, rewriting of global to shared memory copy by CUDA-specific `cp.async` can be made simpler if the rewrite happens after buffer flattening and loop vectorization passes.
+
+### `wait(in-flight-count)` vs `wait(finished-count)`
+
+ It would be more intuitive if the semantics of `wait(N)` was “Wait until the oldest N async operations have completed”. But that would make translation to the corresponding PTX instruction more complicated, since we additionally need to keep track of the “number of async operations in-flight” at each synchronization point, and make that an additional argument to `async_wait_stage` so that we can do subtraction during translation of `async_wait_stage` to `cp.async`.
+
+One of the pros of `wait(in-flight-count)` semantics is that, it is trivial to let all in-flight async operations to complete: `wait(0)`. The alternative semantics would require, again, precise tracking of the number of async operations in-flight at the desired sync point.
+
+
+### More examples
+
+**Three stages of compute, where the first two stages are async**. The second stage is both an async producer and consumer. This example demonstrates the use of the “stage” parameter. Note that there is no distinction of asynchronous copy or compute.
+
+```python
+B = alloc([1])
+C = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[0] = B[0] + 1
+    D[i] = C[0] + 1
+```
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1, 2])
+sch.annotate(i, "software_pipeline_order", [0, 1, 2])
+# The first and second statements are async, and they are in different stages
+sch.annotate(i, "software_pipeline_async_stages", [0, 1])
+```
+
+```python
+B = alloc([2])
+C = alloc([2])
+
+# Prologue
+for i in range(2):
+   async_scope:
+      B[i % 2]  = A[i] + 1
+   async_commit_stage(0)
+
+   if 1 <= i:
+      async_wait_stage(0, 1)
+      async_scope:
+         C[(i - 1) % 2] = B[(i - 1) % 2] + 1
+      async_commit_stage(1)
+
+# Body
+for i in range(14):
+   # Stage 0
+   async_scope:
+      B[(i + 2) % 2]  = A[i + 2] + 1
+   async_commit_stage(0)
+
+   # Stage 1
+   async_wait_stage(0, 1)
+   async_scope:
+      C[(i + 1) % 2] = B[(i + 1) % 2] + 1
+   async_commit_stage(1)
+
+   # Stage 2
+   async_wait_stage(1, 1)
+   D[i] = C[i % 2] + 1
+
+
+# Epilogue
+for i in range(2):
+   if i < 1:
+     async_wait_stage(0, 0)
+     async_scope:
+        C[(i + 15) % 2] = B[(i + 15) % 2] + 1
+     async_commit_stage(1)
+
+   if i < 1:
+      async_wait_group(1, 1)
+   else:
+      async_wait_group(1, 0)
+
+   D[(i + 14) % 2] = C[(i + 14) % 2] + 1
+
+```
+
+**Multi-stage pipelined GEMM where the shared memory copy is 4x multi-buffered + async, and shared to local copy is double-buffered**. This example uses a highly non-obvious annotation below and exercises the nested pipelining feature in the TIR software pipeline transform.
+
+```python
+sch.annotate(k0, ann_key="software_pipeline_stage", ann_val=[0, 0, 2, 3, 3])
+sch.annotate(k0, ann_key="software_pipeline_order", ann_val=[0, 1, 3, 2, 4])
+sch.annotate(k0, ann_key="software_pipeline_async_stages", ann_val=[0, 1])
+
+sch.annotate(k1, ann_key="software_pipeline_stage", ann_val=[0, 0, 1])
+sch.annotate(k1, ann_key="software_pipeline_order", ann_val=[0, 1, 2])
+```
+
+`async_commit_stage` is inserted after copies to `A_shared` and `B_shared` are issued, so that the two copies can be awaited as one chunk.
+
+```python
+
+# Prologue
+A_local = [2, ...]
+B_local = [2, ...]
+A_shared = [4, ...]
+B_shared = [4, ...]
+
+for i in range(3):
+   async_scope:
+     A_shared[i] <- global[...]
+
+   async_scope:
+     B_shared[i] <- global[...]
+
+   async_commit_stage(0)
+
+   if 2 <= i:
+      async_wait_stage(0, 2)
+      A_local[0] <- A_shared[0, ...]
+      B_local[0] <- B_shared[0, ...]
+
+# Body
+for i in range(125):
+   async_scope:
+     A_shared[(i + 3) % 4] <- global[...]
+
+   async_scope:
+     B_shared[(i + 3) % 4] <- global[...]
+
+   async_commit_stage(0)
+
+   async_wait_stage(0, 2)
+
+   A_local[1] <- A_shared[i % 4, ...]
+   B_local[1] <- B_shared[i % 4, ...]
+
+   compute(A_local[0], B_local[0])
+
+   A_local[0] <- A_shared[(i + 1) % 4, ...]
+   B_local[0] <- B_shared[(i + 1) % 4, ...]
+
+   compute(A_local[1], B_local[1])
+
+# Epilogue
+for i in range(3):
+   async_wait_stage(0, 1 - i)
+
+   A_local[1] <- A_shared[0, ...]
+   B_local[1] <- B_shared[0, ...]
+
+   compute(A_local[0], B_local[0])
+
+   if i < 2:
+      A_local[0] <- A_shared[0, ...]
+      B_local[0] <- B_shared[0, ...]
+
+   compute(A_local[1], B_local[1])
+
+```
+
+### Implicit vs explicit approach to synchronization
+
+The model of async synchronization adopted by CUDA can be categorized as an “implicit” one: Instead of saying “Wait for this operation to complete”, it says “Wait until only N most recent async operations are in flight”, or equivalently, “Wait until M oldest async operates have completed”, where M = “number of async operations in flight” - N.
+
+In contrast, a standard and intuitive approach is more explicit, e.g. wait for the operation associated with “this” token / future to complete etc. This is true for “async-await” in general-purpose languages, [Async](https://mlir.llvm.org/docs/Dialects/AsyncDialect/) and [NVGPU](https://mlir.llvm.org/docs/Dialects/NVGPU/) dialects in MLIR, for example.
+
+In general, the explicit approach is probably more preferable, since
+
+- It makes it obvious which operation is waiting on which
+- It is less stateful (less assumption on how the underlying HW should work)
+- It naturally handles synchronization in the presence of control flow (since we can only wait on an operation that has actually happened).
+
+These properties may help if we want do some analysis of async programs.
+
+The current design started from and has stayed with CUDA’s implicit synchronization model based on counting, primarily because it makes mapping to the corresponding PTX instructions trivial. We can adopt the explicit model instead, if we have a good way to translate token-based synchronization to the counting one for PTX. So far, we do not have a good solution for this. MLIR has adopted the token abstraction, but they have not solve this problem either: Their `DeviceAsyncWaitOp` has [an optional attribute `numGroups`](https://mlir.llvm.org/docs/Dialects/NVGPU/#nvgpudevice_async_wait-mlirnvgpudeviceasyncwaitop) that directly corresponds to "in-flight count", and they basically generate either `wait(numGroups)` or `wait(0)`, [in their translation](https://github.com/llvm/llvm-project/blob/main/mlir/lib/Conversion/NVGPUToNVVM/NVGPUToNVVM.cpp#L426-L427) of  `DeviceAsyncWaitOp` (token based) to PTX `cp.async` (counting based). `wait(0)` is always correct but least precise / efficient.
+
+
+
+(The following is highly speculative) On the other hand, translation from “count” to “token” seems more feasible: At each synchronization point, a backend presumably maintains the number and the order of pending async operations. Given the count, it should be possible to derive the correct token from the corresponding ordered list of tokens.
+
+Importantly, we are not trying to propose a “general async semantics to TIR”. Rather the goal is to come up with an async design specifically for the TIR software pipeline transform. Hopefully, this would allow making assumptions that might not be reasonable in more general settings (control flow, relative order of operations etc), and simplify the implementation by building on what the TIR software pipeline already produces as part of transform. Hopefully, reading the explanation below should also convince one that the counting based sync is a natural fit (or “good enough”) for the TIR software pipeline.

Review Comment:
   Since a unique integer "stage" value passed to commit/wait would be equivalent to creating and waiting of a fence, I think the current design does result in general async semantics.  I'm not opposed to doing so, but if we want to avoid a generic async framework, we should make a more restrictive data structure for it.



##########
rfcs/0077-async-pipeline.md:
##########
@@ -0,0 +1,528 @@
+- Feature Name: Asynchronous stage in software pipeline
+- Authors: [Masahiro Masuda](https://github.com/masahi), [Wuwei Lin](https://github.com/vinx13/)
+- Start Date: (2022-06-17)
+
+# Summary
+This RFC proposes two TIR intrinsics and an additional annotation to the TIR software pipeline transform, to express asynchrony **within the device code**.
+Asynchrony is prevalent on the host (runtime) side, and this proposal is the first step toward bringing the notion of an asynchronous operation in the
+generated code.
+
+The most important component we should agree on is the model of synchronization: Coming up with a design that is general enough to be useful for diverse backends, while making sure that the chosen design can be translated to a low-level synchronization model of a particular backend, is highly non-trivial.
+The approach described in this document is motivated by a use case for NVIDIA GPUs, but we took some cares so that the design can be adopted by other backends. For example, if a backend has an asynchronous DMA engine, vector and tensor unit, we can specify that each of them runs asynchronously in different stages in a pipeline, with necessary synchronization between them.
+
+The proposed model may have diverged from conventional ones, but we believe that this is a good fit for the TIR software pipeline specifically.
+
+# Asynchronous stage in a software pipeline
+
+### Background: What is a software pipeline, and what does the TIR software pipeline transform do?
+
+Software pipeline is an optimization technique to improve instruction-level parallelism of a loop. For example, given this program:
+
+```python
+B = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+```
+
+the goal is to overlap the execution of two statements in the loop body, by letting the two statements operate on different iterations of the loop. This way, the second statement would no longer depend on the completion of the first statement in the same iteration.
+
+The TIR software pipeline transform enables such transformation at the TIR level. We annotate the loop in the above program to specify, for each statement in the loop, the “stage” and the “order” in the pipeline:
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+sch.annotate(i, "software_pipeline_order", [0, 1])
+```
+
+Given the annotation above, the TIR software pipeline transform would break up the loop into three parts: prologue, pipeline body and epilogue. Different “stage” in the pipeline body become independent of each other, and the integer value of “stage” tells how many iterations each statement goes ahead of its consumer.
+
+```python
+B = alloc([2])
+
+# Prologue
+B[0] = A[0]
+
+# Body
+for i in range(15):
+    B[(i + 1) % 2] = A[i] + 1
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+C[15] = B[1] + 1
+```
+
+The two statements in the body can potentially run in parallel, if the underlying HW supports out-of-order execution.
+
+### Making parallelism more explicit: Asynchronous pipeline
+
+What’s currently available today is, after all, a “software” pipeline: whether or not independent statements actually run in parallel is up to the underlying HW, and programmers have little control over it. Moreover, for in-order processors like Hexagon DSP, this transformation alone would not help.
+
+The goal of this work is to support HW-backed asynchrony in the pipeline. Asynchronous data movement is becoming increasingly important in GPU computing, and NPUs typically have multiple kinds of asynchronous units (DMA copy, vector & matrix compute etc). To exploit such hardware features, it’s essential that we express all kinds of available asynchronies in the IR.
+
+A user of the TIR software pipeline transform will be able to specify which data movement or compute block should become asynchronous by an additional annotation. For example, given the  annotation specifying that the first block in the pipeline be made async,
+
+```python
+for i in range(16):
+    B[0] = A[i] + 1
+    C[i] = B[0] + 1
+
+...
+sch.annotate(i, "software_pipeline_stage", [0, 1])
+...
+
+# "0" refers to the first element in te list [0, 1] above, i.e. the first block
+sch.annotate(i, "software_pipeline_async_stages", [0])
+```
+
+we generate the following IR. An asynchronous block is decorated with the `async_scope` attribute, and two intrinsics are inserted to express synchronization.
+
+```python
+B = alloc([2])
+
+# Prologue
+async_scope:
+   B[0] = A[0]
+   async_commit_stage(0)
+
+# Body
+for i in range(15):
+    async_scope:
+        B[(i + 1) % 2] = A[i] + 1
+    async_commit_stage(0)
+
+    async_wait_stage(0, 1)
+    C[i] = B[i % 2] + 1
+
+# Epilogue
+async_wait_stage(0, 0)
+C[15] = B[1] + 1
+```
+
+**Semantics of the proposed intrinsics**. “stage” refers to the same notion in the TIR software pipeline.
+- `async_commit_stage(i)` : Group one or more invocation of async operations, and “commit” them to the `i`-th stage. The exact interpretation of “committing” can be up to each backend, but informally it signifies that a group of async operations are now in-flight. The group of operations committed together is awaited as one chunk, and thus they constitute the granularity at which the synchronization intrinsic discussed next operates on.
+- `async_wait_stage(i, N)` : Block until only `N` **most recent** committed groups are still in-flight at the stage `i` . In other words, if there are `M` committed groups in-flight at the stage `i`, at the invocation of `async_wait_stage(i, N)`, `M - N` oldest committed groups would be forced to complete. `N` doesn’t have to be a constant, but some backends may require a constant count (e.g. PTX)
+
+They directly correspond to the async data movement instructions in CUDA (PTX): [`cp.async.commit_group`](https://docs.nvidia.com/cuda/parallel-thread-execution/index.html#data-movement-and-conversion-instructions-cp-async-commit-group) and [`cp.async.wait_group`](https://docs.nvidia.com/cuda/parallel-thread-execution/index.html#data-movement-and-conversion-instructions-cp-async-wait-group).
+
+The CUDA counterparts do not have the notion of “stage”, since there is only one kind of async operation (copy from global to shared memory) supported by the current generation of NVIDIA GPU (Ampere, at the time of writing). To support more general cases where there could be multiple kinds of async “engine”, each of which corresponds to a different stage in an async pipeline, TIR `async_commit_stage` and `async_wait_stage` take a “stage” parameter.
+
+**The role of async_scope**. `async_scope` is represented by `AttrStmt` with key `tir::attr::async_scope`. It is inserted to let later transform passes know that the enclosed statement is intended to run asynchronously. This way, the actual lowering to target-dependent asynchronous instructions
+can happen much later in the compilation flow, rather than before the software pipeline transform using tensorization. For example, rewriting of global to shared memory copy by CUDA-specific `cp.async` can be made simpler if the rewrite happens after buffer flattening and loop vectorization passes.
+
+### `wait(in-flight-count)` vs `wait(finished-count)`
+
+ It would be more intuitive if the semantics of `wait(N)` was “Wait until the oldest N async operations have completed”. But that would make translation to the corresponding PTX instruction more complicated, since we additionally need to keep track of the “number of async operations in-flight” at each synchronization point, and make that an additional argument to `async_wait_stage` so that we can do subtraction during translation of `async_wait_stage` to `cp.async`.
+
+One of the pros of `wait(in-flight-count)` semantics is that, it is trivial to let all in-flight async operations to complete: `wait(0)`. The alternative semantics would require, again, precise tracking of the number of async operations in-flight at the desired sync point.
+
+
+### More examples
+
+**Three stages of compute, where the first two stages are async**. The second stage is both an async producer and consumer. This example demonstrates the use of the “stage” parameter. Note that there is no distinction of asynchronous copy or compute.
+
+```python
+B = alloc([1])
+C = alloc([1])
+
+for i in range(16):
+    B[0] = A[i] + 1
+    C[0] = B[0] + 1
+    D[i] = C[0] + 1
+```
+
+```python
+sch = ...
+sch.annotate(i, "software_pipeline_stage", [0, 1, 2])
+sch.annotate(i, "software_pipeline_order", [0, 1, 2])
+# The first and second statements are async, and they are in different stages
+sch.annotate(i, "software_pipeline_async_stages", [0, 1])
+```
+
+```python
+B = alloc([2])
+C = alloc([2])
+
+# Prologue
+for i in range(2):
+   async_scope:
+      B[i % 2]  = A[i] + 1
+   async_commit_stage(0)
+
+   if 1 <= i:
+      async_wait_stage(0, 1)
+      async_scope:
+         C[(i - 1) % 2] = B[(i - 1) % 2] + 1
+      async_commit_stage(1)
+
+# Body
+for i in range(14):
+   # Stage 0
+   async_scope:
+      B[(i + 2) % 2]  = A[i + 2] + 1
+   async_commit_stage(0)
+
+   # Stage 1
+   async_wait_stage(0, 1)
+   async_scope:
+      C[(i + 1) % 2] = B[(i + 1) % 2] + 1
+   async_commit_stage(1)
+
+   # Stage 2
+   async_wait_stage(1, 1)
+   D[i] = C[i % 2] + 1
+
+
+# Epilogue
+for i in range(2):
+   if i < 1:
+     async_wait_stage(0, 0)
+     async_scope:
+        C[(i + 15) % 2] = B[(i + 15) % 2] + 1
+     async_commit_stage(1)
+
+   if i < 1:
+      async_wait_group(1, 1)
+   else:
+      async_wait_group(1, 0)
+
+   D[(i + 14) % 2] = C[(i + 14) % 2] + 1
+
+```
+
+**Multi-stage pipelined GEMM where the shared memory copy is 4x multi-buffered + async, and shared to local copy is double-buffered**. This example uses a highly non-obvious annotation below and exercises the nested pipelining feature in the TIR software pipeline transform.
+
+```python
+sch.annotate(k0, ann_key="software_pipeline_stage", ann_val=[0, 0, 2, 3, 3])
+sch.annotate(k0, ann_key="software_pipeline_order", ann_val=[0, 1, 3, 2, 4])
+sch.annotate(k0, ann_key="software_pipeline_async_stages", ann_val=[0, 1])
+
+sch.annotate(k1, ann_key="software_pipeline_stage", ann_val=[0, 0, 1])
+sch.annotate(k1, ann_key="software_pipeline_order", ann_val=[0, 1, 2])
+```
+
+`async_commit_stage` is inserted after copies to `A_shared` and `B_shared` are issued, so that the two copies can be awaited as one chunk.
+
+```python
+
+# Prologue
+A_local = [2, ...]
+B_local = [2, ...]
+A_shared = [4, ...]
+B_shared = [4, ...]
+
+for i in range(3):
+   async_scope:
+     A_shared[i] <- global[...]
+
+   async_scope:
+     B_shared[i] <- global[...]
+
+   async_commit_stage(0)
+
+   if 2 <= i:
+      async_wait_stage(0, 2)
+      A_local[0] <- A_shared[0, ...]
+      B_local[0] <- B_shared[0, ...]
+
+# Body
+for i in range(125):
+   async_scope:
+     A_shared[(i + 3) % 4] <- global[...]
+
+   async_scope:
+     B_shared[(i + 3) % 4] <- global[...]
+
+   async_commit_stage(0)
+
+   async_wait_stage(0, 2)
+
+   A_local[1] <- A_shared[i % 4, ...]
+   B_local[1] <- B_shared[i % 4, ...]
+
+   compute(A_local[0], B_local[0])
+
+   A_local[0] <- A_shared[(i + 1) % 4, ...]
+   B_local[0] <- B_shared[(i + 1) % 4, ...]
+
+   compute(A_local[1], B_local[1])
+
+# Epilogue
+for i in range(3):
+   async_wait_stage(0, 1 - i)
+
+   A_local[1] <- A_shared[0, ...]
+   B_local[1] <- B_shared[0, ...]
+
+   compute(A_local[0], B_local[0])
+
+   if i < 2:
+      A_local[0] <- A_shared[0, ...]
+      B_local[0] <- B_shared[0, ...]
+
+   compute(A_local[1], B_local[1])
+
+```
+
+### Implicit vs explicit approach to synchronization
+
+The model of async synchronization adopted by CUDA can be categorized as an “implicit” one: Instead of saying “Wait for this operation to complete”, it says “Wait until only N most recent async operations are in flight”, or equivalently, “Wait until M oldest async operates have completed”, where M = “number of async operations in flight” - N.
+
+In contrast, a standard and intuitive approach is more explicit, e.g. wait for the operation associated with “this” token / future to complete etc. This is true for “async-await” in general-purpose languages, [Async](https://mlir.llvm.org/docs/Dialects/AsyncDialect/) and [NVGPU](https://mlir.llvm.org/docs/Dialects/NVGPU/) dialects in MLIR, for example.
+
+In general, the explicit approach is probably more preferable, since
+
+- It makes it obvious which operation is waiting on which
+- It is less stateful (less assumption on how the underlying HW should work)
+- It naturally handles synchronization in the presence of control flow (since we can only wait on an operation that has actually happened).
+
+These properties may help if we want do some analysis of async programs.
+
+The current design started from and has stayed with CUDA’s implicit synchronization model based on counting, primarily because it makes mapping to the corresponding PTX instructions trivial. We can adopt the explicit model instead, if we have a good way to translate token-based synchronization to the counting one for PTX. So far, we do not have a good solution for this. MLIR has adopted the token abstraction, but they have not solve this problem either: Their `DeviceAsyncWaitOp` has [an optional attribute `numGroups`](https://mlir.llvm.org/docs/Dialects/NVGPU/#nvgpudevice_async_wait-mlirnvgpudeviceasyncwaitop) that directly corresponds to "in-flight count", and they basically generate either `wait(numGroups)` or `wait(0)`, [in their translation](https://github.com/llvm/llvm-project/blob/main/mlir/lib/Conversion/NVGPUToNVVM/NVGPUToNVVM.cpp#L426-L427) of  `DeviceAsyncWaitOp` (token based) to PTX `cp.async` (counting based). `wait(0)` is always correct but least precise / efficient.
+
+
+
+(The following is highly speculative) On the other hand, translation from “count” to “token” seems more feasible: At each synchronization point, a backend presumably maintains the number and the order of pending async operations. Given the count, it should be possible to derive the correct token from the corresponding ordered list of tokens.

Review Comment:
   Though speculative, this makes sense to me.  This could even be done within the TIR itself, with each "stage" having a unique integer value, and only ever using `N=0` when waiting.  In effect, the integer passed to commit/wait would be the unique token.  (This assumes that there is minimal overhead in maintaining the existence of a "stage" that can be waited on.)
   
   If sequentially assigned, I think this would also allow the token-based integer synchronization to be translated into the count-based synchronization.  (e.g. If iteration `i` launches stage `i + offset` and wants `N` in-flight, it could wait on stage `i + offset - N`.)



-- 
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: commits-unsubscribe@tvm.apache.org

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