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/11/10 16:27:53 UTC

[GitHub] [tvm] Icemist opened a new pull request, #13349: Add vtcm memory capacity verification for Hexagon target

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

   
   The main items that have been added are:
   
   tvm.tir.analysis.calculate_allocated_bytes() 
   > to calculate allocated memory per memory scope
   
   tir.transform.VerifyVTCMLimit(limit)
   
   > to verify if the size of the allocated vtcm memory satisfies the limit
   
   tvm.target.hexagon().vtcm_capacity 
   
   > attribute to pass the limit
   
   tir.vtcm_capacity 
   
   > context configuration attribute to pass the limit alternatively
   
   Interaction with tuners:
   - autotvm - In the case of the rejection of all trials falls within the implementation
   - auto_scheduler - сan fall into an infinite loop when all variants are rejected, requires a limit on the number of errors in the search phase, the same as in the measurement phase
   - meta_scheduler - need to add a test


-- 
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] csullivan commented on pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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

   Thank you @Icemist, @adstraw, @echuraev @janetsc, @masahi for the great discussion and changes, this is 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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/target/utils.h:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 target/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_TARGET_UTILS_H_
+#define TVM_TARGET_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/optional.h>
+#include <tvm/target/target.h>
+
+namespace tvm {
+
+/*! \brief Extract attribute from a target. */
+inline Integer Extract(const Target& target, const char* name) {

Review Comment:
   I also had a thought that the attributes may not be integer, but I suggest we leave this for another patch or when needed.



-- 
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 #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/target/target.py:
##########
@@ -642,6 +646,8 @@ def hexagon(cpu_ver="v66", **kwargs):
         Whether to use IEEE HVX instructions
     num_cores : int (default: 4)
         The number of HVX threads. This attribute is required by meta scheduler.
+    vtcm_capacity: int (default: 0)
+        Hexagon VTCM capacity limitation. Value of 0 disables this limitation.

Review Comment:
   Do you mean, "If the value is 0, the capacity is treated as unbounded"?



-- 
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] csullivan merged pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.
+ */
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = target->GetAttr<Integer>("vtcm-capacity").value().IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();

Review Comment:
   To move this RP forward. I've separated this part into another [branch](https://github.com/apache/tvm/commit/9a14eee9fd05abc11800f15bedae3d6e1d5008d6), let's look at it in more detail there.
   



-- 
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] csullivan commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+#include <tvm/target/utils.h>
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = Extract(target, "vtcm-capacity").IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();
+          // Convert Function to IRModule
+          transform::PassContext pass_ctx = transform::PassContext::Current();
+          tir::PrimFunc f = WithAttr(GetRef<tir::PrimFunc>(prim_func), "global_symbol",
+                                     runtime::String(g_var->name_hint));
+          bool noalias = pass_ctx->GetConfig<Bool>("tir.noalias", Bool(true)).value();
+          if (noalias) {
+            f = WithAttr(std::move(f), "tir.noalias", Bool(true));
+          }
+          IRModule mod = IRModule(Map<GlobalVar, BaseFunc>({{GlobalVar(g_var->name_hint), f}}));
+          lowered = tvm::transform::Sequential(pass_list)(std::move(mod));
+        } catch (const dmlc::Error& e) {
+          return false;
+        }
+        if (!Verify(lowered)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  Postproc Clone() const {
+    ObjectPtr<VerifyVTCMLimitNode> n = make_object<VerifyVTCMLimitNode>(*this);
+    n->vtcm_capacity = this->vtcm_capacity;
+    return Postproc(n);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.VerifyVTCMLimit";

Review Comment:
   USMP is a whole program analysis, whereas the VerifyVTCMLimit pass is a single op/primfunc analysis. In principle to achieve what you are suggesting @janetsc we would want to update the MS preprocessor to be replaced with a preprocessor _generator_ which can generate a preprocessor to use based on the live VTCM as calculated by USMP. This should be quite doable in Relax. 
   
   As a side note, with the introduction of whole program memory planning, the phase ordering of memory planning and tuning can become complex. 



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/rpc/tracker.py:
##########
@@ -348,7 +348,7 @@ def close(self, conn):
         if "key" in conn._info:
             for value in conn.put_values:
                 _, _, _, key = value
-                rpc_key = key.split(":")[0]
+                rpc_key = key.rsplit(":", 1)[0]

Review Comment:
   Moved into https://github.com/apache/tvm/pull/13420



-- 
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] janetsc commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
tests/python/contrib/test_hexagon/test_cache_read_write.py:
##########
@@ -222,5 +223,28 @@ def _visit(stmt):
     )
 
 
+def _raises_exception(f):
+    try:
+        f()
+    except Exception:
+        return True
+    return False
+
+
+@pytest.mark.parametrize("vtcm_capacity,limited", [(65536, False), (8192, True)])
+def test_vtcm_limit(vtcm_capacity, limited):
+    """Test lowering with vtcm mem scope"""
+    mod = tvm.IRModule.from_expr(scale_by_two.with_attr("global_symbol", "main"))
+    sch = tir.Schedule(mod, debug_mask="all")
+    block_c = sch.get_block("C")
+    (flat,) = sch.get_loops(block_c)
+    outer, _, _, _ = sch.split(flat, factors=[8, 4, 2, 128])
+    cache_block = sch.cache_read(block_c, 0, storage_scope="global.vtcm")
+    sch.compute_at(cache_block, outer)
+
+    with tvm.transform.PassContext(config={"tir.vtcm_capacity": vtcm_capacity}):

Review Comment:
   It would be interesting in the future if we could pass this capacity down to the device API for two reasons - to verify that it is possible, and to attempt to allocate only that size from QURT in HexagonVtcmPool



##########
tests/python/unittest/test_tir_analysis_calculate_allocated_memory.py:
##########
@@ -0,0 +1,65 @@
+# 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.
+import pytest
+
+import tvm
+from tvm import tir
+from tvm.script import tir as T
+
+
+# fmt: off
+@T.prim_func
+def matmul_some_scope(a: T.handle, b: T.handle, c: T.handle) -> None:
+    A = T.match_buffer(a, [128, 128], scope="global")
+    A_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    B = T.match_buffer(b, [128, 128], scope="global")
+    B_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    C = T.match_buffer(c, [128, 128], scope="global")
+    C_ = T.alloc_buffer([128, 128], dtype="float32", scope="global")
+    
+    for i, j in T.grid(128, 128):
+        with T.block("A_.texture"):
+            A_[i, j] = A[i, j]
+    for i, j in T.grid(128, 128):
+        with T.block("B_.texture"):
+            B_[i, j] = B[i, j]
+
+    for i, j, k in T.grid(128, 128, 128):
+        with T.block("update"):
+            vi, vj, vk = T.axis.remap("SSR", [i, j, k])
+            with T.init():
+                C_[vi, vj] = 0.0
+            C_[vi, vj] = C_[vi, vj] + A_[vi, vk] * B_[vj, vk]
+    
+    for i, j in T.grid(128, 128):
+        with T.block("C"):
+            C[i, j] = C_[i, j]
+# fmt: on
+
+
+@pytest.mark.parametrize(
+    "scope,size", [("global", 131072), ("global.texture", 131072*2), ("global.texture-nhwc", 0)]
+)
+def test_allocates(scope, size):

Review Comment:
   Can you add a test specifically to make sure the calculated VTCM size matches what we expect?  (To make sure it picks up "global.vtcm", as the LowerVtcmAlloc pass alters those nodes.)



-- 
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] echuraev commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/driver/driver_api.h:
##########
@@ -78,20 +78,22 @@ TVM_DLL transform::Sequential HostModulePassManager(IRModule mixed_mod, Target t
  * \brief Lower an IRModule (optimize with it with the pass list defined in CreatePassList)
  * \param mod The IRmodule to lower
  * \param simple_mode Disables the loop partition pass. Defaults to false.
+ * \param target The device target to provide additional characteristics. Defaults to not defined.
  * \return The result module.
  */
-TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false);
+TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false, Target target = {});

Review Comment:
   I have some doubts about introducing the new parameter with target to many lowering functions. As far as I understood, the only one thing why we need to have this `target` parameter is for passing value of `vtcm_capacity` to the lowering functions.
   From my point of view, we shouldn't modify this common API. As for me, the better way is to pass this option through `PassContext` (e.g. in the same way it works for `relay.ToMixedPrecision.keep_orig_output_dtype`).
   
   @tqchen, @csullivan, @junrushao, @tkonolige, In file [test_vtcm.py](https://github.com/apache/tvm/pull/13349/files#diff-9380a067fbc7e640cb0257906fadaf5c400f07f46500d530af7c601b29855eb6R83-R95) @Icemist has presented possible ways for passing value of `vtcm_capacity`:
   1. By passing target as a parameter to the `lower` function: 
       ```python
       _raises_exception(lambda: tvm.lower(sch.mod["main"], target=target)) == limited
       ``` 
   2. By using `with` statement:
       ```python
       with target:
           assert (
               _raises_exception(lambda: tvm.lower(sch.mod["main"])) == limited
           ), "VTCM memory allocation limiter does not work correctly "
       ```
   3. By passing value of `vtcm_capacity` to `PassContext`:
       ```python
       with tvm.transform.PassContext(config={"tir.vtcm_capacity": vtcm_capacity}):
           assert (
               _raises_exception(lambda: tvm.lower(sch.mod["main"])) == limited
           ), "VTCM memory allocation limiter does not work correctly "
       ```
       
   Personally, I prefer the third option, because in this case, I believe, it is not necessary to modify the API of lowering functions. What do you think about that?



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

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

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


[GitHub] [tvm] csullivan commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/driver/driver_api.h:
##########
@@ -78,20 +78,22 @@ TVM_DLL transform::Sequential HostModulePassManager(IRModule mixed_mod, Target t
  * \brief Lower an IRModule (optimize with it with the pass list defined in CreatePassList)
  * \param mod The IRmodule to lower
  * \param simple_mode Disables the loop partition pass. Defaults to false.
+ * \param target The device target to provide additional characteristics. Defaults to not defined.
  * \return The result module.
  */
-TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false);
+TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false, Target target = {});

Review Comment:
   For now we should avoid making an API change to the compiler that would introduce `target` to lowering without some community discussion on the topic.
   
   That said I also prefer deriving this target specific information from the target. In this case if we make the following change to move LowerVTCMAlloc into the MixedModulePassManager, we can add `tir::transform::VerifyVTCMLimit` just prior to it in the `mixed_pass_list`. As these passes are run at build time (after lowering), the target is available for use. 
   
   ```diff
   diff --git a/src/driver/driver_api.cc b/src/driver/driver_api.cc
   index e5e3998b1..babac2bf6 100644
   --- a/src/driver/driver_api.cc
   +++ b/src/driver/driver_api.cc
   @@ -225,10 +225,8 @@ Array<tvm::transform::Pass> CreatePassList(bool disable_loop_partition) {
      if (!disable_storage_rewrite) {
        pass_list.push_back(tir::transform::StorageRewrite());
      }
   -  // LowerVtcmAlloc must occur after any transformations that modify memory allocation locations
   -  pass_list.push_back(tir::transform::LowerVtcmAlloc());
   -  bool use_async_copy = pass_ctx->GetConfig<Bool>("tir.use_async_copy", Bool(false)).value();
   
   +  bool use_async_copy = pass_ctx->GetConfig<Bool>("tir.use_async_copy", Bool(false)).value();
      if (use_async_copy) {
        pass_list.push_back(tir::transform::LowerAsyncDMA());
      }
   @@ -539,6 +537,9 @@ transform::Sequential MixedModulePassManager(IRModule mixed_mod, Target target)
   
      Array<Pass> mixed_pass_list;
   +  mixed_pass_list.push_back(tir::transform::VerifyVTCMLimit(GetVTCMCapacity(target, pass_ctx)));
   +  // LowerVtcmAlloc must occur after any transformations that modify memory allocation locations
   +  mixed_pass_list.push_back(tir::transform::LowerVtcmAlloc());
   +
      mixed_pass_list.push_back(tir::transform::BindTarget(target));
   
      mixed_pass_list.push_back(tir::transform::VerifyMemory());
   ```
   



-- 
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] janetsc commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+#include <tvm/target/utils.h>
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = Extract(target, "vtcm-capacity").IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();
+          // Convert Function to IRModule
+          transform::PassContext pass_ctx = transform::PassContext::Current();
+          tir::PrimFunc f = WithAttr(GetRef<tir::PrimFunc>(prim_func), "global_symbol",
+                                     runtime::String(g_var->name_hint));
+          bool noalias = pass_ctx->GetConfig<Bool>("tir.noalias", Bool(true)).value();
+          if (noalias) {
+            f = WithAttr(std::move(f), "tir.noalias", Bool(true));
+          }
+          IRModule mod = IRModule(Map<GlobalVar, BaseFunc>({{GlobalVar(g_var->name_hint), f}}));
+          lowered = tvm::transform::Sequential(pass_list)(std::move(mod));
+        } catch (const dmlc::Error& e) {
+          return false;
+        }
+        if (!Verify(lowered)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  Postproc Clone() const {
+    ObjectPtr<VerifyVTCMLimitNode> n = make_object<VerifyVTCMLimitNode>(*this);
+    n->vtcm_capacity = this->vtcm_capacity;
+    return Postproc(n);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.VerifyVTCMLimit";

Review Comment:
   That means we might be discarding viable options.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/autotvm/measure/measure_methods.py:
##########
@@ -493,7 +495,9 @@ def set_task(self, task):
         return server, tracker
 
 
-def _build_func_common(measure_input, runtime=None, check_gpu=None, build_option=None):
+def _build_func_common(
+    measure_input, runtime=None, check_gpu=None, build_option=None, check_hexagon=None

Review Comment:
   done.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/driver/driver_api.h:
##########
@@ -78,20 +78,22 @@ TVM_DLL transform::Sequential HostModulePassManager(IRModule mixed_mod, Target t
  * \brief Lower an IRModule (optimize with it with the pass list defined in CreatePassList)
  * \param mod The IRmodule to lower
  * \param simple_mode Disables the loop partition pass. Defaults to false.
+ * \param target The device target to provide additional characteristics. Defaults to not defined.
  * \return The result module.
  */
-TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false);
+TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false, Target target = {});

Review Comment:
   In the case of a refusal to change the api and using only tir.vtcm_capacity as a transmitter - 
   
   In tvm.build we can have this logic:
   ```python
   vtcm_capacity = target.vtcm_capacity if target.kind.name == "hexagon" else 0
   with tvm.transform.PassContext(config={"tir.vtcm_capacity": vtcm_capacity}):
   	# code uses lowering
   	
   ```
   to leave the API as pleasant as possible:
   ```python
   target = tvm.target.hexagon("v68", vtcm_capacity=128)
   target = tvm.target.Target(target, host=target)
   func = tvm.build(sch, [], target, name="func")
   ```
   
   But also anticipate that it will be necessary to find and test/modify additional functions including C++ and python parts that use the lowering for the hexagon target.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/target/target.py:
##########
@@ -642,6 +646,8 @@ def hexagon(cpu_ver="v66", **kwargs):
         Whether to use IEEE HVX instructions
     num_cores : int (default: 4)
         The number of HVX threads. This attribute is required by meta scheduler.
+    vtcm_capacity: int (default: 0)
+        Hexagon VTCM capacity limitation. Value of 0 disables this limitation.

Review Comment:
   changed.



-- 
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] echuraev commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/target/utils.h:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 target/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_TARGET_UTILS_H_
+#define TVM_TARGET_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/optional.h>
+#include <tvm/target/target.h>
+
+namespace tvm {
+
+/*! \brief Extract attribute from a target. */
+inline Integer Extract(const Target& target, const char* name) {
+  ICHECK(target.defined());
+  if (Optional<Integer> v = target->GetAttr<Integer>(name)) {
+    return v.value();
+  }
+  LOG(FATAL) << "AttributedError: \"" << name << "\" is not defined in the target";
+  throw;

Review Comment:
   As far as I remember, `LOG(FATAL)` should throw an exception. So, we don't need to add `throw` after that. If there is a compilation warning about return value, probably we could do something like that:
   ```suggestion
     Optional<Integer> v = target->GetAttr<Integer>(name);
     ICHECK(v != nullptr) << "AttributedError: \"" << name << "\" is not defined in the target";
     return v.value();
   ```



##########
python/tvm/driver/build_module.py:
##########
@@ -221,29 +221,30 @@ def build(
     ----
     See the note on :any:`tvm.target` on target string format.
     """
-    if isinstance(inputs, te.Schedule):
-        if args is None:
-            raise ValueError("args must be given for build from schedule")
-        input_mod = lower(inputs, args, name=name, binds=binds)
-    elif isinstance(inputs, (list, tuple, container.Array)):
-        merged_mod = tvm.IRModule({})
-        for x in inputs:
-            merged_mod.update(lower(x))
-        input_mod = merged_mod
-    elif isinstance(inputs, PrimFunc):
-        input_mod = lower(inputs, name=name)
-    elif isinstance(inputs, tvm.IRModule):
-        input_mod = lower(inputs)
-    elif not isinstance(inputs, (dict, container.Map)):
-        raise ValueError(
-            f"Inputs must be te.Schedule, IRModule, PrimFunc, "
-            f"or dict of target to IRModule, "
-            f"but got {type(inputs)}."
-        )
+    target = Target.current() if target is None else target
+    target = target if target else "llvm"
+    with target:

Review Comment:
   Sorry, could you please explain, why do we need this target context and these changes?



##########
include/tvm/target/utils.h:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 target/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_TARGET_UTILS_H_
+#define TVM_TARGET_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/optional.h>
+#include <tvm/target/target.h>
+
+namespace tvm {
+
+/*! \brief Extract attribute from a target. */
+inline Integer Extract(const Target& target, const char* name) {

Review Comment:
   The extracted attributes might have different types, not only `Integer`. Probably we can do this function as a template? Also, have some doubts about do we really need this helper function?



##########
python/tvm/autotvm/measure/measure_methods.py:
##########
@@ -493,7 +495,9 @@ def set_task(self, task):
         return server, tracker
 
 
-def _build_func_common(measure_input, runtime=None, check_gpu=None, build_option=None):
+def _build_func_common(
+    measure_input, runtime=None, check_gpu=None, build_option=None, check_hexagon=None

Review Comment:
   Probably we could pass all check arguments (such as `check_gpu` and `check_hexagon`) in a dictionary or list? In this case it won't be necessary to change function signature every time when we want to add new check



-- 
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] Icemist commented on pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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

   CC @masahi @csullivan


-- 
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] tvm-bot commented on pull request #13349: Add vtcm memory capacity verification for Hexagon target

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

   <!---bot-comment-->
   
   Thanks for contributing to TVM! Please refer to the contributing guidelines https://tvm.apache.org/docs/contribute/ for useful information and tips. Please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @-ing them in a comment.
   
   
   
   <sub>Generated by [tvm-bot](https://github.com/apache/tvm/blob/main/ci/README.md#github-actions)</sub>


-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/target/utils.h:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 target/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_TARGET_UTILS_H_
+#define TVM_TARGET_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/optional.h>
+#include <tvm/target/target.h>
+
+namespace tvm {
+
+/*! \brief Extract attribute from a target. */
+inline Integer Extract(const Target& target, const char* name) {

Review Comment:
   deleted.



##########
include/tvm/target/utils.h:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 target/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_TARGET_UTILS_H_
+#define TVM_TARGET_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/optional.h>
+#include <tvm/target/target.h>
+
+namespace tvm {
+
+/*! \brief Extract attribute from a target. */
+inline Integer Extract(const Target& target, const char* name) {
+  ICHECK(target.defined());
+  if (Optional<Integer> v = target->GetAttr<Integer>(name)) {
+    return v.value();
+  }
+  LOG(FATAL) << "AttributedError: \"" << name << "\" is not defined in the target";
+  throw;

Review Comment:
   deleted.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/driver/driver_api.h:
##########
@@ -78,20 +78,22 @@ TVM_DLL transform::Sequential HostModulePassManager(IRModule mixed_mod, Target t
  * \brief Lower an IRModule (optimize with it with the pass list defined in CreatePassList)
  * \param mod The IRmodule to lower
  * \param simple_mode Disables the loop partition pass. Defaults to false.
+ * \param target The device target to provide additional characteristics. Defaults to not defined.
  * \return The result module.
  */
-TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false);
+TVM_DLL IRModule LowerModule(IRModule mod, bool simple_mode = false, Target target = {});

Review Comment:
   That's the way I like it. My tests are passed with this change.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
include/tvm/target/utils.h:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 target/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_TARGET_UTILS_H_
+#define TVM_TARGET_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/optional.h>
+#include <tvm/target/target.h>
+
+namespace tvm {
+
+/*! \brief Extract attribute from a target. */
+inline Integer Extract(const Target& target, const char* name) {

Review Comment:
   This is not a new function, but a relocation of an existing one, to be used in other code blocks without duplication.



-- 
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] csullivan commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.
+ */
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = target->GetAttr<Integer>("vtcm-capacity").value().IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();

Review Comment:
   We'll need to include a set of passes here that reproduces the standard lowering in TIR including the new `VerifyVTCMLimit` in order to actually exclude bad samples in the search. 



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/driver/build_module.py:
##########
@@ -221,29 +221,30 @@ def build(
     ----
     See the note on :any:`tvm.target` on target string format.
     """
-    if isinstance(inputs, te.Schedule):
-        if args is None:
-            raise ValueError("args must be given for build from schedule")
-        input_mod = lower(inputs, args, name=name, binds=binds)
-    elif isinstance(inputs, (list, tuple, container.Array)):
-        merged_mod = tvm.IRModule({})
-        for x in inputs:
-            merged_mod.update(lower(x))
-        input_mod = merged_mod
-    elif isinstance(inputs, PrimFunc):
-        input_mod = lower(inputs, name=name)
-    elif isinstance(inputs, tvm.IRModule):
-        input_mod = lower(inputs)
-    elif not isinstance(inputs, (dict, container.Map)):
-        raise ValueError(
-            f"Inputs must be te.Schedule, IRModule, PrimFunc, "
-            f"or dict of target to IRModule, "
-            f"but got {type(inputs)}."
-        )
+    target = Target.current() if target is None else target
+    target = target if target else "llvm"
+    with target:

Review Comment:
   It was necessary to use with statement to be able to get the target object using Target::current().
   Now I have changed this by passing the target as an optional parameter for lower function.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.
+ */
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = target->GetAttr<Integer>("vtcm-capacity").value().IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();

Review Comment:
   To move this RP forward. I've separated this part into another [branch](https://github.com/Deelvin/tvm/pull/new/avoronov/verify_vtcm_limit_postproc), let's look at it in more detail there.
   



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/relay/backend/te_compiler.cc:
##########
@@ -444,8 +445,8 @@ class TECompilerImpl : public TECompilerNode {
       }
       auto func_name = value->cached_func->prim_fn_var->name_hint;
       VLOG(1) << "scheduling";
-      IRModule scheduled_module = tvm::LowerSchedule(value->cached_func->schedule, all_args,
-                                                     func_name, binds, global_var_supply);
+      IRModule scheduled_module = tvm::LowerSchedule(
+          value->cached_func->schedule, all_args, func_name, binds, global_var_supply, false, {});

Review Comment:
   ```suggestion
             value->cached_func->schedule, all_args, func_name, binds, global_var_supply, false, key->target);
   ```
   change if the targeting logic stays.



-- 
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 #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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

   cc @kparzysz-quic @janetsc 


-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
tests/python/unittest/test_tir_analysis_calculate_allocated_memory.py:
##########
@@ -0,0 +1,65 @@
+# 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.
+import pytest
+
+import tvm
+from tvm import tir
+from tvm.script import tir as T
+
+
+# fmt: off
+@T.prim_func
+def matmul_some_scope(a: T.handle, b: T.handle, c: T.handle) -> None:
+    A = T.match_buffer(a, [128, 128], scope="global")
+    A_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    B = T.match_buffer(b, [128, 128], scope="global")
+    B_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    C = T.match_buffer(c, [128, 128], scope="global")
+    C_ = T.alloc_buffer([128, 128], dtype="float32", scope="global")
+    
+    for i, j in T.grid(128, 128):
+        with T.block("A_.texture"):
+            A_[i, j] = A[i, j]
+    for i, j in T.grid(128, 128):
+        with T.block("B_.texture"):
+            B_[i, j] = B[i, j]
+
+    for i, j, k in T.grid(128, 128, 128):
+        with T.block("update"):
+            vi, vj, vk = T.axis.remap("SSR", [i, j, k])
+            with T.init():
+                C_[vi, vj] = 0.0
+            C_[vi, vj] = C_[vi, vj] + A_[vi, vk] * B_[vj, vk]
+    
+    for i, j in T.grid(128, 128):
+        with T.block("C"):
+            C[i, j] = C_[i, j]
+# fmt: on
+
+
+@pytest.mark.parametrize(
+    "scope,size", [("global", 131072), ("global.texture", 131072*2), ("global.texture-nhwc", 0)]
+)
+def test_allocates(scope, size):

Review Comment:
   I meant comparing the memory counted with pass and how much memory is allocated with nd_mem_alloc_with_scope after lowering.
   Something like(a sketch): 
    ```
      def calc(irmod):
           size = 0
           def _visit(stmt):
               nonlocal size
               if isinstance(stmt, tvm.tir.expr.Call) and isinstance(stmt.op, tvm.ir.op.Op) and stmt.op.name == "tir.nd_mem_alloc_with_scope":
                   storage_scope, extents_size, call = stmt.args
                   extents, = call.args
                   dtype = call.dtype
                   print("ATTRS", storage_scope, extents_size, extents, dtype)
                   if storage_scope == "global.vtcm":
                       size += extents_size * dtype.bytes() #
   
           tvm.tir.stmt_functor.post_order_visit(irmod["main"].body, _visit)
           return size
   
       print(calc(lowered))
   ```



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/meta_schedule/tir_integration.py:
##########
@@ -117,7 +120,7 @@ def tune_tir(
         task_weights=[1.0],
         work_dir=work_dir,
         max_trials_global=max_trials_global,
-        max_trials_per_task=max_trials_global,
+        max_trials_per_task=max_trials_per_task,

Review Comment:
   Yes, mb it is redundant, but it unifies the interface. I will remove it from this PR.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/meta_schedule/tir_integration.py:
##########
@@ -117,7 +120,7 @@ def tune_tir(
         task_weights=[1.0],
         work_dir=work_dir,
         max_trials_global=max_trials_global,
-        max_trials_per_task=max_trials_global,
+        max_trials_per_task=max_trials_per_task,

Review Comment:
   removed.



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
tests/python/unittest/test_tir_analysis_calculate_allocated_memory.py:
##########
@@ -0,0 +1,65 @@
+# 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.
+import pytest
+
+import tvm
+from tvm import tir
+from tvm.script import tir as T
+
+
+# fmt: off
+@T.prim_func
+def matmul_some_scope(a: T.handle, b: T.handle, c: T.handle) -> None:
+    A = T.match_buffer(a, [128, 128], scope="global")
+    A_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    B = T.match_buffer(b, [128, 128], scope="global")
+    B_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    C = T.match_buffer(c, [128, 128], scope="global")
+    C_ = T.alloc_buffer([128, 128], dtype="float32", scope="global")
+    
+    for i, j in T.grid(128, 128):
+        with T.block("A_.texture"):
+            A_[i, j] = A[i, j]
+    for i, j in T.grid(128, 128):
+        with T.block("B_.texture"):
+            B_[i, j] = B[i, j]
+
+    for i, j, k in T.grid(128, 128, 128):
+        with T.block("update"):
+            vi, vj, vk = T.axis.remap("SSR", [i, j, k])
+            with T.init():
+                C_[vi, vj] = 0.0
+            C_[vi, vj] = C_[vi, vj] + A_[vi, vk] * B_[vj, vk]
+    
+    for i, j in T.grid(128, 128):
+        with T.block("C"):
+            C[i, j] = C_[i, j]
+# fmt: on
+
+
+@pytest.mark.parametrize(
+    "scope,size", [("global", 131072), ("global.texture", 131072*2), ("global.texture-nhwc", 0)]
+)
+def test_allocates(scope, size):

Review Comment:
   I have now updated the tests.
   tests/python/contrib/test_hexagon/test_cache_read_write.py::test_vtcm_limit
   tests/python/unittest/test_tir_analysis_calculate_allocated_memory.py::test_scale_by
   tests/python/unittest/test_tir_analysis_calculate_allocated_memory.py::test_matmul_mix_scope
   
   On your question: You mean tests that show how much memory was allocated through nd_mem_alloc_with_scope after lower, and compare this to what tir.analysis.calculate_allocated_bytes shows. Correct?
   
   Our interest in the line - 
   `let buffer_a_global.vtcm: Pointer(global.vtcm int8) = @tir.nd_mem_alloc_with_scope("global.vtcm", 1i64, @tir.tvm_stack_make_shape(1024, dtype=handle), dtype=handle)`
   in the example:
   ```
   @main = primfn(buffer_a_handle: handle, buffer_c_handle: handle) -> ()
     attr = {"global_symbol": "main", "tir.noalias": True}
     buffers = {buffer_a: Buffer(buffer_a_1: Pointer(global int8), int8, [8192], []),
                buffer_c: Buffer(buffer_c_1: Pointer(global int8), int8, [8192], [])}
     buffer_map = {buffer_a_handle: buffer_a, buffer_c_handle: buffer_c}
     preflattened_buffer_map = {buffer_c_handle: buffer_c_2: Buffer(buffer_c_1, int8, [8192], []), buffer_a_handle: buffer_a_2: Buffer(buffer_a_1, int8, [8192], [])} {
     let buffer_a_global.vtcm: Pointer(global.vtcm int8) = @tir.nd_mem_alloc_with_scope("global.vtcm", 1i64, @tir.tvm_stack_make_shape(1024, dtype=handle), dtype=handle)
     for (i_0: int32, 0, 8) {
       for (ax0: int32, 0, 1024) {
         buffer_a_global.vtcm_1: Buffer(buffer_a_global.vtcm, int8, [1024], [], scope="global.vtcm")[ax0] = buffer_a[((i_0*1024) + ax0)]
       }
       for (i_1: int32, 0, 4) {
         for (i_2: int32, 0, 2) {
           for (i_3: int32, 0, 128) {
             let cse_var_2: int32 = (i_1*256)
             let cse_var_1: int32 = (i_2*128)
             buffer_c[((((i_0*1024) + cse_var_2) + cse_var_1) + i_3)] = (buffer_a_global.vtcm_1[((cse_var_2 + cse_var_1) + i_3)]*2i8)
           }
         }
       }
     }
   }
   ```



-- 
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] janetsc commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+#include <tvm/target/utils.h>
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = Extract(target, "vtcm-capacity").IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();
+          // Convert Function to IRModule
+          transform::PassContext pass_ctx = transform::PassContext::Current();
+          tir::PrimFunc f = WithAttr(GetRef<tir::PrimFunc>(prim_func), "global_symbol",
+                                     runtime::String(g_var->name_hint));
+          bool noalias = pass_ctx->GetConfig<Bool>("tir.noalias", Bool(true)).value();
+          if (noalias) {
+            f = WithAttr(std::move(f), "tir.noalias", Bool(true));
+          }
+          IRModule mod = IRModule(Map<GlobalVar, BaseFunc>({{GlobalVar(g_var->name_hint), f}}));
+          lowered = tvm::transform::Sequential(pass_list)(std::move(mod));
+        } catch (const dmlc::Error& e) {
+          return false;
+        }
+        if (!Verify(lowered)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  Postproc Clone() const {
+    ObjectPtr<VerifyVTCMLimitNode> n = make_object<VerifyVTCMLimitNode>(*this);
+    n->vtcm_capacity = this->vtcm_capacity;
+    return Postproc(n);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.VerifyVTCMLimit";

Review Comment:
   Is this guaranteed to run after USMP, if enabled?  If not, we might be eliminating solutions that don't fit within VTCM before USMP (but do afterwards).



-- 
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] Icemist commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/target/target.py:
##########
@@ -642,6 +646,8 @@ def hexagon(cpu_ver="v66", **kwargs):
         Whether to use IEEE HVX instructions
     num_cores : int (default: 4)
         The number of HVX threads. This attribute is required by meta scheduler.
+    vtcm_capacity: int (default: 0)
+        Hexagon VTCM capacity limitation. Value of 0 disables this limitation.

Review Comment:
   True, there is of course another way - 0 also limits, essentially forbidding the use of vtcm. In that case we could use -1 as a disable constraint as well as a default value. But, we don't have such a need right now.



-- 
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 #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/meta_schedule/tir_integration.py:
##########
@@ -117,7 +120,7 @@ def tune_tir(
         task_weights=[1.0],
         work_dir=work_dir,
         max_trials_global=max_trials_global,
-        max_trials_per_task=max_trials_global,
+        max_trials_per_task=max_trials_per_task,

Review Comment:
   If think, for tir tuning, `max_trials_global` is equivalent to `max_trials_per_task`. So we can drop this change (it is unrelated to this PR anyway).



-- 
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 #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
tests/python/contrib/test_hexagon/test_cache_read_write.py:
##########
@@ -194,15 +195,20 @@ def scale_by_two(buffer_a: T.Buffer[(8192,), "int8"], buffer_c: T.Buffer[(8192,)
             buffer_c[i] = buffer_a[i] * T.int8(2)
 
 
-def test_vtcm_lowering():
-    """Test lowering with vtcm mem scope"""
+def get_scale_by_two_schedule():
     mod = tvm.IRModule.from_expr(scale_by_two.with_attr("global_symbol", "main"))
     sch = tir.Schedule(mod, debug_mask="all")
     block_c = sch.get_block("C")
     (flat,) = sch.get_loops(block_c)
     outer, _, _, _ = sch.split(flat, factors=[8, 4, 2, 128])
     cache_block = sch.cache_read(block_c, 0, storage_scope="global.vtcm")
     sch.compute_at(cache_block, outer)
+    return sch
+
+
+def test_vtcm_lowering():

Review Comment:
   Note that this test is migrating to a new file name `test_vtcm.py` in the same directory in PR #13381 



-- 
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] janetsc commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
tests/python/unittest/test_tir_analysis_calculate_allocated_memory.py:
##########
@@ -0,0 +1,65 @@
+# 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.
+import pytest
+
+import tvm
+from tvm import tir
+from tvm.script import tir as T
+
+
+# fmt: off
+@T.prim_func
+def matmul_some_scope(a: T.handle, b: T.handle, c: T.handle) -> None:
+    A = T.match_buffer(a, [128, 128], scope="global")
+    A_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    B = T.match_buffer(b, [128, 128], scope="global")
+    B_ = T.alloc_buffer([128, 128], dtype="float32", scope="global.texture")
+    C = T.match_buffer(c, [128, 128], scope="global")
+    C_ = T.alloc_buffer([128, 128], dtype="float32", scope="global")
+    
+    for i, j in T.grid(128, 128):
+        with T.block("A_.texture"):
+            A_[i, j] = A[i, j]
+    for i, j in T.grid(128, 128):
+        with T.block("B_.texture"):
+            B_[i, j] = B[i, j]
+
+    for i, j, k in T.grid(128, 128, 128):
+        with T.block("update"):
+            vi, vj, vk = T.axis.remap("SSR", [i, j, k])
+            with T.init():
+                C_[vi, vj] = 0.0
+            C_[vi, vj] = C_[vi, vj] + A_[vi, vk] * B_[vj, vk]
+    
+    for i, j in T.grid(128, 128):
+        with T.block("C"):
+            C[i, j] = C_[i, j]
+# fmt: on
+
+
+@pytest.mark.parametrize(
+    "scope,size", [("global", 131072), ("global.texture", 131072*2), ("global.texture-nhwc", 0)]
+)
+def test_allocates(scope, size):

Review Comment:
   Yes, that's what I meant, to make sure it works for VTCM with that lowering pass.  The new tests resolve this - thank you!
   
   For the example you mention, I'm not sure I understand.  Is that the state after lowering, but before calculating VTCM usage?



-- 
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] csullivan commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
src/meta_schedule/postproc/verify_vtcm_limit.cc:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+#include <tvm/target/utils.h>
+#include <tvm/tir/transform.h>
+
+#include "../utils.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief Verify the correctness of the generated Hexagon code. */
+class VerifyVTCMLimitNode : public PostprocNode {
+ public:
+  int64_t vtcm_capacity = 0;
+
+  void InitializeWithTuneContext(const TuneContext& context) final {
+    ICHECK(context->target.defined());
+    Target target = context->target.value();
+    vtcm_capacity = Extract(target, "vtcm-capacity").IntValue();
+  }
+
+  bool Verify(const IRModule& mod) const {
+    for (const auto& kv : mod->functions) {
+      if (auto* n = kv.second.as<tir::PrimFuncNode>()) {
+        auto func = GetRef<tir::PrimFunc>(n);
+        auto sizes = CalculateAllocatedBytes(func);
+        const auto vtcm_allocated = sizes.Get("global.vtcm").value_or(0);
+        if (vtcm_capacity > 0 && vtcm_allocated.IntValue() > vtcm_capacity) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  bool Apply(const tir::Schedule& sch) final {
+    IRModule mod = sch->mod();
+    for (const auto& kv : mod->functions) {
+      const GlobalVar& g_var = kv.first;
+      const BaseFunc& base_func = kv.second;
+      if (const auto* prim_func = base_func.as<tir::PrimFuncNode>()) {
+        IRModule lowered{nullptr};
+        try {
+          auto pass_list = Array<tvm::transform::Pass>();
+          // Convert Function to IRModule
+          transform::PassContext pass_ctx = transform::PassContext::Current();
+          tir::PrimFunc f = WithAttr(GetRef<tir::PrimFunc>(prim_func), "global_symbol",
+                                     runtime::String(g_var->name_hint));
+          bool noalias = pass_ctx->GetConfig<Bool>("tir.noalias", Bool(true)).value();
+          if (noalias) {
+            f = WithAttr(std::move(f), "tir.noalias", Bool(true));
+          }
+          IRModule mod = IRModule(Map<GlobalVar, BaseFunc>({{GlobalVar(g_var->name_hint), f}}));
+          lowered = tvm::transform::Sequential(pass_list)(std::move(mod));
+        } catch (const dmlc::Error& e) {
+          return false;
+        }
+        if (!Verify(lowered)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  Postproc Clone() const {
+    ObjectPtr<VerifyVTCMLimitNode> n = make_object<VerifyVTCMLimitNode>(*this);
+    n->vtcm_capacity = this->vtcm_capacity;
+    return Postproc(n);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.VerifyVTCMLimit";

Review Comment:
   This will be running during a tuning session which will be prior to USMP (and other executor code responsible for model runtime)



-- 
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] echuraev commented on a diff in pull request #13349: [TIR][Hexagon] Add vtcm memory capacity verification for Hexagon target

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


##########
python/tvm/rpc/tracker.py:
##########
@@ -348,7 +348,7 @@ def close(self, conn):
         if "key" in conn._info:
             for value in conn.put_values:
                 _, _, _, key = value
-                rpc_key = key.split(":")[0]
+                rpc_key = key.rsplit(":", 1)[0]

Review Comment:
   Could you please add a test on this change to highlight what is the fix here? Probably `tests/python/unittest/test_runtime_rpc.py` is the right place for such test. Also, maybe it is better to create a separate PR with this fix.



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