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/09/14 23:23:52 UTC

[GitHub] [tvm] adstraw opened a new pull request, #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

adstraw opened a new pull request, #12785:
URL: https://github.com/apache/tvm/pull/12785

   Creates a 2-stage pipeline (DMA, compute) on Hexagon by lowering Async TIR primitives (see [RFC](https://github.com/apache/tvm-rfcs/blob/main/rfcs/0077-async-pipeline.md)) with memory copy semantics to builtin functions for DMA `Copy` and `Wait`.  These builtins can then be lowered through the device API, with example lowering on Hexagon to the `HexagonUserDMA` class.
   
   The example test case contained in this PR creates a 2 stage pipeline which lowers to the following TIR which is built and run on Hexagon.  The _prologue_ contains a DMA `Copy` for the first input.  The _body_ contains a DMA `Copy` for the next input, a DMA `Wait` for the current input and then compute (increment) on the current input.  The _epilogue_ computes the last input.
   
   ```
   # from tvm.script import tir as T
   @tvm.script.ir_module
   class Module:
       @T.prim_func
       def main(A: T.Buffer[2048, "uint8"], B: T.Buffer[2048, "uint8"]) -> None:
           # function attr dict
           T.func_attr({"global_symbol": "main", "tir.noalias": True})
           T.preflattened_buffer(A, [16, 128], dtype="uint8", data=A.data)
           T.preflattened_buffer(B, [16, 128], dtype="uint8", data=B.data)
           # prologue
           A_global = T.allocate([256], "uint8", "global")
           A_global_1 = T.buffer_decl([256], dtype="uint8", data=A_global)
           T.evaluate(T.dma_copy(0, T.address_of(A_global_1[0], dtype="handle"), T.address_of(A[0], dtype="handle"), 128, dtype="int32"))
           # body
           for i in T.serial(15):
               T.evaluate(T.dma_copy(0, T.address_of(A_global_1[(i + 1) % 2 * 128], dtype="handle"), T.address_of(A[i * 128 + 128], dtype="handle"), 128, dtype="int32"))
               T.evaluate(T.dma_wait(0, 1, dtype="int32"))
               for j in T.serial(128):
                   B[i * 128 + j] = A_global_1[i % 2 * 128 + j] + T.uint8(1)
           # epilogue
           T.evaluate(T.dma_wait(0, 0, dtype="int32"))
           for j in T.serial(128):
               B[j + 1920] = A_global_1[j + 128] + T.uint8(1)
   ```


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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974381412


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());

Review Comment:
   I am doing `ICHECK` for things I think should be invariant based on the previous checks.  In this case, a `BufferLoadNode` should contain a buffer as `BufferNode` and, given that we have already checked that it has a single index, the buffer should be contiguous.



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974742950


##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   @masahi what do you think about generalizing that switch by calling it `use_async_copy` and having both the CUDA pass and my new `LowerAsyncDMA` pass use the same switch?



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974420521


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {

Review Comment:
   Added debug message.



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


[GitHub] [tvm] adstraw commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1252585829

   @tvm-bot rerun


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


[GitHub] [tvm] tmoreau89 merged pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
tmoreau89 merged PR #12785:
URL: https://github.com/apache/tvm/pull/12785


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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974420889


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());

Review Comment:
   Added comments to clarify why I am asserting the buffer is contiguous.



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974421108


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {

Review Comment:
   Added debug messages.



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r971383716


##########
src/driver/driver_api.cc:
##########
@@ -225,6 +225,7 @@ Array<tvm::transform::Pass> CreatePassList(bool disable_loop_partition) {
   }
   // LowerVtcmAlloc must occur after any transformations that modify memory allocation locations
   pass_list.push_back(tir::transform::LowerVtcmAlloc());
+  pass_list.push_back(tir::transform::LowerAsyncDMA());

Review Comment:
   Also realized that I may need to have a device specific (Hexagon) enable here and/or TIR annotation or some other mechanism to enable this pass to avoid breaking existing async pipeline implementations e.g. cuda.



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


[GitHub] [tvm] adstraw commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1249542837

   @tvm-bot rerun


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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974519785


##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   I am open to making it a Hexagon only pass, but not quite sure how to do it.  Investigating.



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r972287621


##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   Right, I have attempted to make the pass / intrinsics generic.  Perhaps that is not correct.  We could leave as-is given CI is passing.  Or, we could do _one or both_ of the following:
   
   1) Make the pass / intrinsics specific to Hexagon
   2) Add user level control to opt in to running this pass



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


[GitHub] [tvm] masahi commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
masahi commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r975232657


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        LOG(INFO) << "AsyncDMALowerer exiting because the queue ID observed in the "

Review Comment:
   Please replace all `LOG(INFO)` in this file with `DLOG(INFO)`



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974382751


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());
+
+      // we will be replacing the entire for loop including its index
+      // with a DMA copy instrinsic that spans the entire index space of the for loop
+      // so we will need to repace the for loop index with value zero in the buffer indices

Review Comment:
   That's correct.  Clarified the 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] TejashShah commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
TejashShah commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1248335776

   cc @kparzysz-quic @jverma-quic 


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


[GitHub] [tvm] masahi commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
masahi commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r971701650


##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   I think it is fine to have Hexagon-specific intrinsics here, rather than trying to be more generic than necessary.
   
   For CUDA we have specific copy / wait intrins in https://github.com/apache/tvm/blob/c4aab62c6d588ed06f60b0db9691271f5883dc66/include/tvm/tir/builtin.h#L635-L652  



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


[GitHub] [tvm] tmoreau89 commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
tmoreau89 commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1252820665

   Thanks @adstraw, @JosephTheOctonaut and @masahi for the PR and the reviews, the PR has been merged!


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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974382751


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());
+
+      // we will be replacing the entire for loop including its index
+      // with a DMA copy instrinsic that spans the entire index space of the for loop
+      // so we will need to repace the for loop index with value zero in the buffer indices

Review Comment:
   That's correct.  I will clarify the 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] adstraw commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1252531723

   @tvm-bot rerun


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


[GitHub] [tvm] adstraw commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1252536908

   @tvm-bot rerun


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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r972196429


##########
src/driver/driver_api.cc:
##########
@@ -225,6 +225,7 @@ Array<tvm::transform::Pass> CreatePassList(bool disable_loop_partition) {
   }
   // LowerVtcmAlloc must occur after any transformations that modify memory allocation locations
   pass_list.push_back(tir::transform::LowerVtcmAlloc());
+  pass_list.push_back(tir::transform::LowerAsyncDMA());

Review Comment:
   Resolving this comment.  Will track this in conversation on the [comment](https://github.com/apache/tvm/pull/12785#discussion_r971701650) from @masahi as it is related.



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r971383191


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {

Review Comment:
   As I posted this PR I realized that the `AsyncDMALowerer` probably needs to keep some memory of the queue IDs where it has found memory copy semantics in the `Copy` transform and require a match on queue ID before running the `Wait` transform.



##########
src/driver/driver_api.cc:
##########
@@ -225,6 +225,7 @@ Array<tvm::transform::Pass> CreatePassList(bool disable_loop_partition) {
   }
   // LowerVtcmAlloc must occur after any transformations that modify memory allocation locations
   pass_list.push_back(tir::transform::LowerVtcmAlloc());
+  pass_list.push_back(tir::transform::LowerAsyncDMA());

Review Comment:
   May need to have a device specific (Hexagon) enable here and/or TIR annotation or some other mechanism to enable this pass to avoid breaking existing async pipeline implementations e.g. cuda.



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


[GitHub] [tvm] JosephTheOctonaut commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
JosephTheOctonaut commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r973390163


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {

Review Comment:
   This check implies that `dma_wait` must always follow the corresponding `dma_copy` in program order. Since the `dma_copy` is asynchronous, it's not strictly necessary for correctness that it show up first.
   
   It seems fine for now to require that the stages show up in a specific order, but an error or debug message might be helpful here to clarify why the transform isn't doing anything.



##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {

Review Comment:
   This check, along with L99 and L105, would have the transform fail silently. A debug message might be helpful to explain why the transform couldn't run.



##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());
+
+      // we will be replacing the entire for loop including its index
+      // with a DMA copy instrinsic that spans the entire index space of the for loop
+      // so we will need to repace the for loop index with value zero in the buffer indices

Review Comment:
   If I understand, the purpose of this is to eliminate the index from the expression so the DMA copy receives the buffer range base address, correct?



##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());

Review Comment:
   Previous checks, if failed, cause the transform to fail gracefully. Why are these fatal failures?



##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   The tension in specific-vs-general intrinsics is trying to guess how "general" this style is. We have a good degree of certainty that the CUDA style is unique to PTX, but I would guess that most platforms with DMA engines could support an async copy() and wait() of this style?
   
   The risk in making it general is that we end up with DMA-enabled targets that need a different version. The risk in making it specific is that we end up with multiple target-specific intrinsics that are essentially identical.



##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   Regardless of the choice there, the pass should probably be Hexagon-specific. I think you had another comment @adstraw about breaking the existing async pipeline lowering --- I agree with that; this transform should be Hexagon-only and probably opt-in, since it's mutually exclusive (at the moment) with other async lowerings and destroys the async annotations.



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


[GitHub] [tvm] masahi commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
masahi commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974611391


##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   For CUDA async copy, I added an option that users need to enable manually:
   
   https://github.com/apache/tvm/blob/036aa722aefd7126c17063cddb260b9347f17598/src/driver/driver_api.cc#L546-L551
   
   For Hexagon, we may just enable this pass in `driver_api.cc` whenever the target is Hexagon.



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


[GitHub] [tvm] masahi commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
masahi commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974757392


##########
include/tvm/tir/builtin.h:
##########
@@ -720,6 +720,16 @@ TVM_DLL const Op& texture2d_load();
  */
 TVM_DLL const Op& mem_copy();
 
+/*!
+ * \brief Initiate a non-blocking DMA copy from source to destination
+ */
+TVM_DLL const Op& dma_copy();
+
+/*!
+ * \brief Wait until the number of DMAs in flight is less than or equal to some maximum
+ */
+TVM_DLL const Op& dma_wait();

Review Comment:
   Yeah I like that. And for Hexagon we can enable that switch by default.



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


[GitHub] [tvm] adstraw commented on a diff in pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
adstraw commented on code in PR #12785:
URL: https://github.com/apache/tvm/pull/12785#discussion_r974381412


##########
src/tir/transforms/lower_async_dma.cc:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file lower_async_dma.cc
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+class AsyncDMALowerer : public StmtExprMutator {
+ public:
+  AsyncDMALowerer() {}
+
+  Stmt VisitStmt_(const AttrStmtNode* op) final {
+    // Convert this, for example:
+    // attr [0] "async_wait_queue_scope" = 0;
+    // attr [0] "async_wait_inflight_count" = 0;
+    //
+    // To this:
+    // @tir.dma_wait(
+    //   0, /* queue id */
+    //   0, /* in flight count */
+    //   dtype=int32
+    // )
+    if (op->attr_key == tir::attr::async_wait_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // abort if we have not seen this queue ID in `copy` transform
+      if (queue_ids.find(queue_id) == queue_ids.end()) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      auto async_wait = op->body.as<AttrStmtNode>();
+      ICHECK(async_wait && async_wait->attr_key == tir::attr::async_wait_inflight_count);
+
+      auto call_dma_wait =
+          Evaluate(Call(DataType::Int(32), builtin::dma_wait(), {queue_id, async_wait->value}));
+
+      // concatenate the call with the body and return
+      return SeqStmt({call_dma_wait, async_wait->body});
+
+      // Convert this, for example:
+      // attr [0] "async_commit_queue_scope" = 0;
+      // attr [0] "async_scope" = 1;
+      // for (ax0: int32, 0, 128) {
+      //   A_global[ax0] = A[ax0]
+      // }
+      //
+      // To this:
+      // @tir.dma_copy(
+      //   0, /* queue id */
+      //   @tir.address_of(A_global[0], dtype=handle),
+      //   @tir.address_of(A[0], dtype=handle),
+      //   128, /* size */
+      //   dtype=int32
+      // )
+    } else if (op->attr_key == tir::attr::async_commit_queue_scope) {
+      // get queue ID
+      auto queue_id_node = op->value.as<IntImmNode>();
+      ICHECK(queue_id_node);
+      int queue_id = queue_id_node->value;
+
+      // save queue ID for inspection in `wait` transform
+      queue_ids.insert(queue_id);
+
+      // walk the graph to verify this is a mem copy ...
+      // 1) async_commit_queue_scope contains async_scope
+      auto async_scope = op->body.as<AttrStmtNode>();
+      ICHECK(async_scope && async_scope->attr_key == tir::attr::async_scope);
+
+      // 2) async_scope contains single for loop
+      auto for_loop = async_scope->body.as<ForNode>();
+      if (!for_loop) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 3) for loop contains buffer store with single index
+      auto bufferstorenode = for_loop->body.as<BufferStoreNode>();
+      if (!bufferstorenode || bufferstorenode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // 4) buffer store value is a buffer load with single index
+      auto bufferloadnode = bufferstorenode->value.as<BufferLoadNode>();
+      if (!bufferloadnode || bufferloadnode->indices.size() != 1) {
+        return StmtExprMutator::VisitStmt_(op);
+      }
+
+      // get store buffer and assert that it is contiguous
+      auto bufferstore = bufferstorenode->buffer.as<BufferNode>();
+      ICHECK(bufferstore && bufferstore->strides.empty());
+
+      // get load buffer and assert that it is contiguous
+      auto bufferload = bufferloadnode->buffer.as<BufferNode>();
+      ICHECK(bufferload && bufferload->strides.empty());

Review Comment:
   I am doing `ICHECK` for things I think should be invariant based on the previous checks.  In this case, a `BufferLoadNode` should have a buffer and, given that we have already checked that it has a single index, the buffer should be contiguous.



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


[GitHub] [tvm] github-actions[bot] commented on pull request #12785: [Hexagon] 2-Stage Pipeline; Lower Async TIR primitives to Hexagon User DMA

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #12785:
URL: https://github.com/apache/tvm/pull/12785#issuecomment-1252588590

   Failed to re-run CI in https://github.com/apache/tvm/actions/runs/3091706323
   
   <details>
   
   ```
   Traceback (most recent call last):
     File "/home/runner/work/tvm/tvm/ci/scripts/git_utils.py", line 121, in _request
       with request.urlopen(req, data) as response:
     File "/usr/lib/python3.8/urllib/request.py", line 222, in urlopen
       return opener.open(url, data, timeout)
     File "/usr/lib/python3.8/urllib/request.py", line 531, in open
       response = meth(req, response)
     File "/usr/lib/python3.8/urllib/request.py", line 640, in http_response
       response = self.parent.error(
     File "/usr/lib/python3.8/urllib/request.py", line 569, in error
       return self._call_chain(*args)
     File "/usr/lib/python3.8/urllib/request.py", line 502, in _call_chain
       result = func(*args)
     File "/usr/lib/python3.8/urllib/request.py", line 649, in http_error_default
       raise HTTPError(req.full_url, code, msg, hdrs, fp)
   urllib.error.HTTPError: HTTP Error 403: Forbidden
   
   During handling of the above exception, another exception occurred:
   
   Traceback (most recent call last):
     File "ci/scripts/github_tvmbot.py", line 574, in comment_failure
       raise item
     File "ci/scripts/github_tvmbot.py", line 685, in run
       pr.rerun_github_actions()
     File "ci/scripts/github_tvmbot.py", line 563, in rerun_github_actions
       raise e
     File "ci/scripts/github_tvmbot.py", line 555, in rerun_github_actions
       actions_github.post(f"actions/runs/{workflow_id}/rerun-failed-jobs", data={})
     File "/home/runner/work/tvm/tvm/ci/scripts/git_utils.py", line 143, in post
       return self._request(self.base + url, data, method="POST")
     File "/home/runner/work/tvm/tvm/ci/scripts/git_utils.py", line 126, in _request
       raise RuntimeError(f"Error response: {msg}\n{error_data}")
   RuntimeError: Error response: HTTP Error 403: Forbidden
   {"message":"This workflow is already running","documentation_url":"https://docs.github.com/rest/reference/actions#re-run-workflow-failed-jobs"}
   
   ```
   
   </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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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