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 2021/11/30 11:53:01 UTC

[GitHub] [tvm] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759198212



##########
File path: tests/python/unittest/test_tir_usmp_algo.py
##########
@@ -0,0 +1,492 @@
+# 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, script
+from tvm.script import tir as T
+from tvm.tir import stmt_functor
+from tvm.tir.usmp import utils as usmp_utils
+from tvm.target import Target
+
+
+def _replace_stmt_with_buf_var_names(buffer_info_map):
+    """helper to replace tir.allocates with buffer names"""
+    new_buffer_info_map = dict()
+    for k, v in buffer_info_map.items():
+        new_buffer_info_map[v.buffer_var.name] = k
+    return new_buffer_info_map
+
+
+def _verify_conflicts(main_buf_name, conflicting_buf_names, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    buf_info = buffer_info_map[main_buf_name]
+    for conflict in buf_info.conflicts:
+        assert conflict.name_hint in conflicting_buf_names
+
+
+def _get_allocates(primfunc):
+    """helper to extract all allocate nodes by name"""
+    allocates = dict()
+
+    def get_allocate(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            allocates[str(stmt.buffer_var.name)] = stmt
+
+    stmt_functor.post_order_visit(primfunc.body, get_allocate)
+    return allocates
+
+
+def assign_poolinfos_to_allocates_in_primfunc(primfunc, pool_infos):

Review comment:
       Ack

##########
File path: tests/python/unittest/test_tir_usmp_algo.py
##########
@@ -0,0 +1,492 @@
+# 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, script
+from tvm.script import tir as T
+from tvm.tir import stmt_functor
+from tvm.tir.usmp import utils as usmp_utils
+from tvm.target import Target
+
+
+def _replace_stmt_with_buf_var_names(buffer_info_map):
+    """helper to replace tir.allocates with buffer names"""
+    new_buffer_info_map = dict()
+    for k, v in buffer_info_map.items():
+        new_buffer_info_map[v.buffer_var.name] = k
+    return new_buffer_info_map
+
+
+def _verify_conflicts(main_buf_name, conflicting_buf_names, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    buf_info = buffer_info_map[main_buf_name]
+    for conflict in buf_info.conflicts:
+        assert conflict.name_hint in conflicting_buf_names
+
+
+def _get_allocates(primfunc):
+    """helper to extract all allocate nodes by name"""
+    allocates = dict()
+
+    def get_allocate(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            allocates[str(stmt.buffer_var.name)] = stmt
+
+    stmt_functor.post_order_visit(primfunc.body, get_allocate)
+    return allocates
+
+
+def assign_poolinfos_to_allocates_in_primfunc(primfunc, pool_infos):
+    """helper to assing poolinfos to allocate nodes in a tir.PrimFunc"""
+
+    def set_poolinfos(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            return tvm.tir.Allocate(
+                buffer_var=stmt.buffer_var,
+                dtype=stmt.dtype,
+                extents=stmt.extents,
+                condition=stmt.condition,
+                body=stmt.body,
+                annotations={tvm.tir.usmp.utils.CANDIDATE_MEMORY_POOL_ATTR: pool_infos},
+            )
+
+    return primfunc.with_body(stmt_functor.ir_transform(primfunc.body, None, set_poolinfos))
+
+
+def assign_poolinfos_to_allocates_in_irmodule(mod, pool_infos):
+    """helper to assing poolinfos to allocate nodes in a IRModule"""
+    ret = tvm.IRModule()
+    for global_var, basefunc in mod.functions.items():
+        if isinstance(basefunc, tvm.tir.PrimFunc):
+            ret[global_var] = assign_poolinfos_to_allocates_in_primfunc(basefunc, pool_infos)
+    return ret
+
+
+def _assign_targets_to_primfuncs_irmodule(mod, target):
+    """helper to assign target for PrimFunc in a IRModule"""
+    ret = tvm.IRModule()
+    for global_var, basefunc in mod.functions.items():
+        if isinstance(basefunc, tvm.tir.PrimFunc):
+            ret[global_var] = basefunc.with_attr("target", target)
+    return ret
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    assert max_workspace_size == size
+
+
+# fmt: off
+@tvm.script.ir_module
+class LinearStructure:
+    @T.prim_func
+    def tvmgen_default_fused_cast_subtract(placeholder_2: T.handle, placeholder_3: T.handle, T_subtract: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_cast_subtract", "tir.noalias": True})
+        placeholder_4 = T.match_buffer(placeholder_2, [1, 224, 224, 3], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        placeholder_5 = T.match_buffer(placeholder_3, [], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        T_subtract_1 = T.match_buffer(T_subtract, [1, 224, 224, 3], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        # body
+        for ax0_ax1_fused_1 in T.serial(0, 224):
+            for ax2_1, ax3_inner_1 in T.grid(224, 3):
+                T.store(T_subtract_1.data, (((ax0_ax1_fused_1*672) + (ax2_1*3)) + ax3_inner_1), (T.cast(T.load("uint8", placeholder_4.data, (((ax0_ax1_fused_1*672) + (ax2_1*3)) + ax3_inner_1)), "int16") - T.load("int16", placeholder_5.data, 0)), True)
+
+    @T.prim_func
+    def tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast(placeholder_62: T.handle, placeholder_63: T.handle, placeholder_64: T.handle, T_cast_20: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast", "tir.noalias": True})
+        placeholder_65 = T.match_buffer(placeholder_62, [1, 224, 224, 3], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        placeholder_66 = T.match_buffer(placeholder_63, [7, 7, 3, 64], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        placeholder_67 = T.match_buffer(placeholder_64, [1, 1, 1, 64], dtype="int32", elem_offset=0, align=128, offset_factor=1)
+        T_cast_21 = T.match_buffer(T_cast_20, [1, 112, 112, 64], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        # body
+        PaddedInput_7 = T.allocate([157323], "int16", "global")
+        for i0_i1_fused_7 in T.serial(0, 229):
+            for i2_7, i3_7 in T.grid(229, 3):
+                T.store(PaddedInput_7, (((i0_i1_fused_7*687) + (i2_7*3)) + i3_7), T.if_then_else(((((2 <= i0_i1_fused_7) and (i0_i1_fused_7 < 226)) and (2 <= i2_7)) and (i2_7 < 226)), T.load("int16", placeholder_65.data, ((((i0_i1_fused_7*672) + (i2_7*3)) + i3_7) - 1350)), T.int16(0), dtype="int16"), True)
+        for ax0_ax1_fused_ax2_fused_7 in T.serial(0, 12544):
+            Conv2dOutput_7 = T.allocate([64], "int32", "global")
+            for ff_3 in T.serial(0, 64):
+                T.store(Conv2dOutput_7, ff_3, 0, True)
+                for ry_2, rx_2, rc_7 in T.grid(7, 7, 3):
+                    T.store(Conv2dOutput_7, ff_3, (T.load("int32", Conv2dOutput_7, ff_3) + (T.cast(T.load("int16", PaddedInput_7, (((((T.floordiv(ax0_ax1_fused_ax2_fused_7, 112)*1374) + (ry_2*687)) + (T.floormod(ax0_ax1_fused_ax2_fused_7, 112)*6)) + (rx_2*3)) + rc_7)), "int32")*T.cast(T.load("int16", placeholder_66.data, ((((ry_2*1344) + (rx_2*192)) + (rc_7*64)) + ff_3)), "int32"))), True)
+            for ax3_inner_7 in T.serial(0, 64):
+                T.store(T_cast_21.data, ((ax0_ax1_fused_ax2_fused_7*64) + ax3_inner_7), T.cast(T.max(T.min(T.q_multiply_shift((T.load("int32", Conv2dOutput_7, ax3_inner_7) + T.load("int32", placeholder_67.data, ax3_inner_7)), 1939887962, 31, -9, dtype="int32"), 255), 0), "uint8"), True)
+
+    @T.prim_func
+    def tvmgen_default_fused_nn_max_pool2d_cast(placeholder_28: T.handle, T_cast_6: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_nn_max_pool2d_cast", "tir.noalias": True})
+        placeholder_29 = T.match_buffer(placeholder_28, [1, 112, 112, 64], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        T_cast_7 = T.match_buffer(T_cast_6, [1, 56, 56, 64], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        # body
+        tensor_2 = T.allocate([200704], "uint8", "global")
+        for ax0_ax1_fused_4 in T.serial(0, 56):
+            for ax2_4 in T.serial(0, 56):
+                for ax3_init in T.serial(0, 64):
+                    T.store(tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_init), T.uint8(0), True)
+                for rv0_rv1_fused_1, ax3_2 in T.grid(9, 64):
+                    T.store(tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_2), T.max(T.load("uint8", tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_2)), T.if_then_else(((((ax0_ax1_fused_4*2) + T.floordiv(rv0_rv1_fused_1, 3)) < 112) and (((ax2_4*2) + T.floormod(rv0_rv1_fused_1, 3)) < 112)), T.load("uint8", placeholder_29.data, (((((ax0_ax1_fused_4*14336) + (T.floordiv(rv0_rv1_fused_1, 3)*7168)) + (ax2_4*128)) + (T.floormod(rv0_rv1_fused_1, 3)*64)) + ax3_2)), T.uint8(0), dtype="uint8")), True)
+        for ax0_ax1_fused_5 in T.serial(0, 56):
+            for ax2_5, ax3_3 in T.grid(56, 64):
+                T.store(T_cast_7.data, (((ax0_ax1_fused_5*3584) + (ax2_5*64)) + ax3_3), T.cast(T.load("uint8", tensor_2, (((ax0_ax1_fused_5*3584) + (ax2_5*64)) + ax3_3)), "int16"), True)
+
+    @T.prim_func
+    def run_model(input: T.handle, output: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_run_model", "runner_function": True})
+        # body
+        T.attr("default", "device_id", 0)
+        T.attr("default", "device_type", 1)
+        sid_9 = T.allocate([301056], "int8", "global")
+        sid_8 = T.allocate([802816], "int8", "global")
+        T.evaluate(T.call_extern("tvmgen_default_fused_cast_subtract", input, T.lookup_param("p0", dtype="handle"), sid_9, dtype="int32"))
+        T.evaluate(T.call_extern("tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast", sid_9, T.lookup_param("p1", dtype="handle"), T.lookup_param("p2", dtype="handle"), sid_8, dtype="int32"))
+        T.evaluate(T.call_extern("tvmgen_default_fused_nn_max_pool2d_cast", sid_8, output, dtype="int32"))
+    __tvm_meta__ = None
+# fmt: on
+
+
+def print_conflicts(buffer_info_map):
+    """_verify_conflicts("sid_8", ["Conv2dOutput_7", "tensor_2"], buffer_info_map)"""
+
+    for buffer_info_name, buf_info in buffer_info_map.items():
+        conflict_str = "["
+        for conflict in buf_info.conflicts:
+            conflict_str += f'"{conflict.name_hint}", '
+        conflict_str += "]"
+        print(f'_verify_conflicts("{buffer_info_name}", {conflict_str}, buffer_info_map_names)')
+
+
+@pytest.mark.parametrize(
+    ["algorithm", "fast_memory_size", "slow_memory_size"],
+    [("greedy_by_size", 200704, 1418528), ("greedy_by_conflicts", 200704, 1418528)],
+)
+def test_linear(algorithm, fast_memory_size, slow_memory_size):

Review comment:
       The unit testing in this level is to make sure the component is functional and ensure the contract of the component. As long as greedy algorithm goes covering a linear and fan-out case covers most aspect conflict graph one could end up.
   My personal preference would be to check coverage at the integration level of the USMP using these algorithm rather than using hand crafted TIR snippets at this level. WDYT ?
   
   Alternatively, if you had some smaller cases in mind that you'd like tested here, please do share -- so we could consider adding them here.

##########
File path: src/tir/usmp/algo/greedy.cc
##########
@@ -0,0 +1,235 @@
+/*
+ * 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 tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief This source contains greedy algorithms for planning
+ * memory for USMP. There are two algorithms present here :
+ * 1) greedy_by_size and 2) greedy_by_conflicts.
+ *
+ * greedy_by_size : this algorithm prioritizes placing the
+ * largest size buffer to the given pools. The BufferInfo objects
+ * are sorted based on the size and placed on each pool adhering
+ * to size_hint constraint.
+ *
+ * greedy_by_conflicts : this algorithm prioritizes placing the
+ * the most liveness conflicted buffer to the given pools. The
+ * BufferInfo objects are sorted based on the number of conflicts
+ * and placed on each pool adhering to size_hint constraint.
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+
+/*!
+ * \brief This is the base class for Greedy Algorithms where the sorting
+ * is specialized in the extended classes based on the greedy criteria.
+ */
+class GreedyBase {
+ public:
+  GreedyBase() {}
+  /*!
+   * \brief This function should be implemented by the extended classes to sort the BufferInfo
+   * objects based on a criteria and then calling PostSortAllocation.
+   */
+  virtual Map<BufferInfo, PoolAllocation> PlanMemory(const Array<BufferInfo>& buffer_info_arr) = 0;
+
+ protected:
+  /*!
+   * \brief Rounds up the offset to satisfy the alignement requirement
+   */
+  size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                    const int& byte_alignment) {
+    return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+  }
+
+  /*!
+   * \brief A helper function check whether a offset is valid given the constraints
+   */
+  bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                        const size_t& size_bytes) {
+    if (candidate_pool->size_hint_bytes == -1) {
+      // this means pool is not bounded
+      return true;
+    }
+    auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+    auto max_address = next_offset + size_bytes;
+    if (max_address <= pool_size) {
+      return true;
+    }
+    return false;
+  }
+
+  /*!
+   * \brief Selects a pool for placement in the given set of ordered pool candidates
+   */
+  PoolInfo SelectPlacementPool(
+      const Array<PoolInfo>& pool_candidates,
+      const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+    // Here the pool candidates are ordered when it is consumed by the algorithm.
+    // This could be from order the user has specified. However, schedulers are
+    // welcome to change the order for performance reasons.
+    for (const auto& pool_info : pool_candidates) {
+      if (pool_offsets.count(pool_info)) {
+        return pool_info;
+      }
+    }
+    ICHECK(false) << "TVM USMP Internal Error: no candidate have been selected!";

Review comment:
       Yea, make sense, will change.

##########
File path: tests/python/unittest/test_tir_usmp_algo.py
##########
@@ -0,0 +1,492 @@
+# 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, script
+from tvm.script import tir as T
+from tvm.tir import stmt_functor
+from tvm.tir.usmp import utils as usmp_utils
+from tvm.target import Target
+
+
+def _replace_stmt_with_buf_var_names(buffer_info_map):
+    """helper to replace tir.allocates with buffer names"""
+    new_buffer_info_map = dict()
+    for k, v in buffer_info_map.items():
+        new_buffer_info_map[v.buffer_var.name] = k
+    return new_buffer_info_map
+
+
+def _verify_conflicts(main_buf_name, conflicting_buf_names, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    buf_info = buffer_info_map[main_buf_name]
+    for conflict in buf_info.conflicts:
+        assert conflict.name_hint in conflicting_buf_names
+
+
+def _get_allocates(primfunc):
+    """helper to extract all allocate nodes by name"""
+    allocates = dict()
+
+    def get_allocate(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            allocates[str(stmt.buffer_var.name)] = stmt
+
+    stmt_functor.post_order_visit(primfunc.body, get_allocate)
+    return allocates
+
+
+def assign_poolinfos_to_allocates_in_primfunc(primfunc, pool_infos):
+    """helper to assing poolinfos to allocate nodes in a tir.PrimFunc"""
+
+    def set_poolinfos(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            return tvm.tir.Allocate(
+                buffer_var=stmt.buffer_var,
+                dtype=stmt.dtype,
+                extents=stmt.extents,
+                condition=stmt.condition,
+                body=stmt.body,
+                annotations={tvm.tir.usmp.utils.CANDIDATE_MEMORY_POOL_ATTR: pool_infos},
+            )
+
+    return primfunc.with_body(stmt_functor.ir_transform(primfunc.body, None, set_poolinfos))
+
+
+def assign_poolinfos_to_allocates_in_irmodule(mod, pool_infos):
+    """helper to assing poolinfos to allocate nodes in a IRModule"""
+    ret = tvm.IRModule()
+    for global_var, basefunc in mod.functions.items():
+        if isinstance(basefunc, tvm.tir.PrimFunc):
+            ret[global_var] = assign_poolinfos_to_allocates_in_primfunc(basefunc, pool_infos)
+    return ret
+
+
+def _assign_targets_to_primfuncs_irmodule(mod, target):
+    """helper to assign target for PrimFunc in a IRModule"""
+    ret = tvm.IRModule()
+    for global_var, basefunc in mod.functions.items():
+        if isinstance(basefunc, tvm.tir.PrimFunc):
+            ret[global_var] = basefunc.with_attr("target", target)
+    return ret
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    assert max_workspace_size == size
+
+
+# fmt: off
+@tvm.script.ir_module
+class LinearStructure:
+    @T.prim_func
+    def tvmgen_default_fused_cast_subtract(placeholder_2: T.handle, placeholder_3: T.handle, T_subtract: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_cast_subtract", "tir.noalias": True})
+        placeholder_4 = T.match_buffer(placeholder_2, [1, 224, 224, 3], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        placeholder_5 = T.match_buffer(placeholder_3, [], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        T_subtract_1 = T.match_buffer(T_subtract, [1, 224, 224, 3], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        # body
+        for ax0_ax1_fused_1 in T.serial(0, 224):
+            for ax2_1, ax3_inner_1 in T.grid(224, 3):
+                T.store(T_subtract_1.data, (((ax0_ax1_fused_1*672) + (ax2_1*3)) + ax3_inner_1), (T.cast(T.load("uint8", placeholder_4.data, (((ax0_ax1_fused_1*672) + (ax2_1*3)) + ax3_inner_1)), "int16") - T.load("int16", placeholder_5.data, 0)), True)
+
+    @T.prim_func
+    def tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast(placeholder_62: T.handle, placeholder_63: T.handle, placeholder_64: T.handle, T_cast_20: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast", "tir.noalias": True})
+        placeholder_65 = T.match_buffer(placeholder_62, [1, 224, 224, 3], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        placeholder_66 = T.match_buffer(placeholder_63, [7, 7, 3, 64], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        placeholder_67 = T.match_buffer(placeholder_64, [1, 1, 1, 64], dtype="int32", elem_offset=0, align=128, offset_factor=1)
+        T_cast_21 = T.match_buffer(T_cast_20, [1, 112, 112, 64], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        # body
+        PaddedInput_7 = T.allocate([157323], "int16", "global")
+        for i0_i1_fused_7 in T.serial(0, 229):
+            for i2_7, i3_7 in T.grid(229, 3):
+                T.store(PaddedInput_7, (((i0_i1_fused_7*687) + (i2_7*3)) + i3_7), T.if_then_else(((((2 <= i0_i1_fused_7) and (i0_i1_fused_7 < 226)) and (2 <= i2_7)) and (i2_7 < 226)), T.load("int16", placeholder_65.data, ((((i0_i1_fused_7*672) + (i2_7*3)) + i3_7) - 1350)), T.int16(0), dtype="int16"), True)
+        for ax0_ax1_fused_ax2_fused_7 in T.serial(0, 12544):
+            Conv2dOutput_7 = T.allocate([64], "int32", "global")
+            for ff_3 in T.serial(0, 64):
+                T.store(Conv2dOutput_7, ff_3, 0, True)
+                for ry_2, rx_2, rc_7 in T.grid(7, 7, 3):
+                    T.store(Conv2dOutput_7, ff_3, (T.load("int32", Conv2dOutput_7, ff_3) + (T.cast(T.load("int16", PaddedInput_7, (((((T.floordiv(ax0_ax1_fused_ax2_fused_7, 112)*1374) + (ry_2*687)) + (T.floormod(ax0_ax1_fused_ax2_fused_7, 112)*6)) + (rx_2*3)) + rc_7)), "int32")*T.cast(T.load("int16", placeholder_66.data, ((((ry_2*1344) + (rx_2*192)) + (rc_7*64)) + ff_3)), "int32"))), True)
+            for ax3_inner_7 in T.serial(0, 64):
+                T.store(T_cast_21.data, ((ax0_ax1_fused_ax2_fused_7*64) + ax3_inner_7), T.cast(T.max(T.min(T.q_multiply_shift((T.load("int32", Conv2dOutput_7, ax3_inner_7) + T.load("int32", placeholder_67.data, ax3_inner_7)), 1939887962, 31, -9, dtype="int32"), 255), 0), "uint8"), True)
+
+    @T.prim_func
+    def tvmgen_default_fused_nn_max_pool2d_cast(placeholder_28: T.handle, T_cast_6: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_nn_max_pool2d_cast", "tir.noalias": True})
+        placeholder_29 = T.match_buffer(placeholder_28, [1, 112, 112, 64], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        T_cast_7 = T.match_buffer(T_cast_6, [1, 56, 56, 64], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        # body
+        tensor_2 = T.allocate([200704], "uint8", "global")
+        for ax0_ax1_fused_4 in T.serial(0, 56):
+            for ax2_4 in T.serial(0, 56):
+                for ax3_init in T.serial(0, 64):
+                    T.store(tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_init), T.uint8(0), True)
+                for rv0_rv1_fused_1, ax3_2 in T.grid(9, 64):
+                    T.store(tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_2), T.max(T.load("uint8", tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_2)), T.if_then_else(((((ax0_ax1_fused_4*2) + T.floordiv(rv0_rv1_fused_1, 3)) < 112) and (((ax2_4*2) + T.floormod(rv0_rv1_fused_1, 3)) < 112)), T.load("uint8", placeholder_29.data, (((((ax0_ax1_fused_4*14336) + (T.floordiv(rv0_rv1_fused_1, 3)*7168)) + (ax2_4*128)) + (T.floormod(rv0_rv1_fused_1, 3)*64)) + ax3_2)), T.uint8(0), dtype="uint8")), True)
+        for ax0_ax1_fused_5 in T.serial(0, 56):
+            for ax2_5, ax3_3 in T.grid(56, 64):
+                T.store(T_cast_7.data, (((ax0_ax1_fused_5*3584) + (ax2_5*64)) + ax3_3), T.cast(T.load("uint8", tensor_2, (((ax0_ax1_fused_5*3584) + (ax2_5*64)) + ax3_3)), "int16"), True)
+
+    @T.prim_func
+    def run_model(input: T.handle, output: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_run_model", "runner_function": True})
+        # body
+        T.attr("default", "device_id", 0)
+        T.attr("default", "device_type", 1)
+        sid_9 = T.allocate([301056], "int8", "global")
+        sid_8 = T.allocate([802816], "int8", "global")
+        T.evaluate(T.call_extern("tvmgen_default_fused_cast_subtract", input, T.lookup_param("p0", dtype="handle"), sid_9, dtype="int32"))
+        T.evaluate(T.call_extern("tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast", sid_9, T.lookup_param("p1", dtype="handle"), T.lookup_param("p2", dtype="handle"), sid_8, dtype="int32"))
+        T.evaluate(T.call_extern("tvmgen_default_fused_nn_max_pool2d_cast", sid_8, output, dtype="int32"))
+    __tvm_meta__ = None
+# fmt: on
+
+
+def print_conflicts(buffer_info_map):
+    """_verify_conflicts("sid_8", ["Conv2dOutput_7", "tensor_2"], buffer_info_map)"""
+
+    for buffer_info_name, buf_info in buffer_info_map.items():
+        conflict_str = "["
+        for conflict in buf_info.conflicts:
+            conflict_str += f'"{conflict.name_hint}", '
+        conflict_str += "]"
+        print(f'_verify_conflicts("{buffer_info_name}", {conflict_str}, buffer_info_map_names)')
+
+
+@pytest.mark.parametrize(
+    ["algorithm", "fast_memory_size", "slow_memory_size"],
+    [("greedy_by_size", 200704, 1418528), ("greedy_by_conflicts", 200704, 1418528)],
+)
+def test_linear(algorithm, fast_memory_size, slow_memory_size):
+    target = Target("c")
+    fast_memory_pool = usmp_utils.PoolInfo(
+        pool_name="fast_memory",
+        target_access={target: usmp_utils.PoolInfo.READ_WRITE_ACCESS},
+        size_hint_bytes=200704,
+    )
+    slow_memory_pool = usmp_utils.PoolInfo(
+        pool_name="slow_memory", target_access={target: usmp_utils.PoolInfo.READ_WRITE_ACCESS}
+    )
+    tir_mod = LinearStructure
+    tir_mod = _assign_targets_to_primfuncs_irmodule(tir_mod, target)
+    tir_mod = assign_poolinfos_to_allocates_in_irmodule(
+        tir_mod, [fast_memory_pool, slow_memory_pool]
+    )
+    main_func = tir_mod["run_model"]
+    buffer_info_map = tvm.tir.usmp.analysis.extract_buffer_info(main_func, tir_mod)
+
+    fcreate_array_bi = tvm.get_global_func("tir.usmp.CreateArrayBufferInfo")
+    buffer_info_arr = fcreate_array_bi(buffer_info_map)
+    fusmp_algo_greedy_by_size = tvm.get_global_func(f"tir.usmp.algo.{algorithm}")
+    buffer_pool_allocations = fusmp_algo_greedy_by_size(buffer_info_arr)

Review comment:
       Thanks! Ack -- will change.

##########
File path: src/tir/usmp/algo/greedy.cc
##########
@@ -0,0 +1,235 @@
+/*
+ * 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 tir/analysis/usmp/algo/greedy_by_size.cc

Review comment:
       Ack

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -415,18 +454,30 @@ Map<BufferInfo, tir::Stmt> BufferInfoExtractor::operator()(const PrimFunc& main_
 
   // Traverse the liveness events using a open set to track what
   // is live while updating the conflicts through out the linear traversal
-  std::unordered_set<BufferInfo, ObjectPtrHash, ObjectPtrEqual> open_set;
+  //  std::unordered_set<BufferInfo, ObjectPtrHash, ObjectPtrEqual> open_set;
+  std::unordered_map<BufferInfo, int, ObjectPtrHash, ObjectPtrEqual> open_set;
   for (const auto& le_event : le_events_timeline) {
     if (le_event.le_type == START) {
-      for (const auto& open_buffer_info : open_set) {
+      for (const auto& kv : open_set) {
+        BufferInfo open_buffer_info = kv.first;
         open_buffer_info->conflicts.push_back(le_event.buffer_info);
         if (le_event.buffer_info != open_buffer_info) {
           le_event.buffer_info->conflicts.push_back(open_buffer_info);
         }
       }
-      open_set.insert(le_event.buffer_info);
+      //      open_set.insert(le_event.buffer_info);

Review comment:
       Ack. Thanks for spotting this!

##########
File path: src/tir/usmp/algo/greedy.cc
##########
@@ -0,0 +1,235 @@
+/*
+ * 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 tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief This source contains greedy algorithms for planning
+ * memory for USMP. There are two algorithms present here :
+ * 1) greedy_by_size and 2) greedy_by_conflicts.
+ *
+ * greedy_by_size : this algorithm prioritizes placing the
+ * largest size buffer to the given pools. The BufferInfo objects
+ * are sorted based on the size and placed on each pool adhering
+ * to size_hint constraint.
+ *
+ * greedy_by_conflicts : this algorithm prioritizes placing the
+ * the most liveness conflicted buffer to the given pools. The
+ * BufferInfo objects are sorted based on the number of conflicts
+ * and placed on each pool adhering to size_hint constraint.
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+
+/*!
+ * \brief This is the base class for Greedy Algorithms where the sorting
+ * is specialized in the extended classes based on the greedy criteria.
+ */
+class GreedyBase {
+ public:
+  GreedyBase() {}
+  /*!
+   * \brief This function should be implemented by the extended classes to sort the BufferInfo
+   * objects based on a criteria and then calling PostSortAllocation.
+   */
+  virtual Map<BufferInfo, PoolAllocation> PlanMemory(const Array<BufferInfo>& buffer_info_arr) = 0;
+
+ protected:
+  /*!
+   * \brief Rounds up the offset to satisfy the alignement requirement
+   */
+  size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                    const int& byte_alignment) {
+    return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+  }
+
+  /*!
+   * \brief A helper function check whether a offset is valid given the constraints
+   */
+  bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                        const size_t& size_bytes) {
+    if (candidate_pool->size_hint_bytes == -1) {
+      // this means pool is not bounded
+      return true;
+    }
+    auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+    auto max_address = next_offset + size_bytes;
+    if (max_address <= pool_size) {
+      return true;
+    }
+    return false;
+  }
+
+  /*!
+   * \brief Selects a pool for placement in the given set of ordered pool candidates
+   */
+  PoolInfo SelectPlacementPool(
+      const Array<PoolInfo>& pool_candidates,
+      const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+    // Here the pool candidates are ordered when it is consumed by the algorithm.
+    // This could be from order the user has specified. However, schedulers are
+    // welcome to change the order for performance reasons.
+    for (const auto& pool_info : pool_candidates) {
+      if (pool_offsets.count(pool_info)) {
+        return pool_info;
+      }
+    }
+    ICHECK(false) << "TVM USMP Internal Error: no candidate have been selected!";
+    return PoolInfo();
+  }
+
+  /*!
+   * \brief This is the base allocation function that works on sorted BufferInfo objects based
+   * on the greedy heuristic. The sorting algorithm has to be called before calling this.
+   */
+  Map<BufferInfo, PoolAllocation> PostSortAllocation(
+      const std::vector<BufferInfo>& buffer_info_vec) {
+    Map<BufferInfo, PoolAllocation> pool_allocations;
+    for (const auto& buf_info : buffer_info_vec) {
+      std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+      for (const auto& pool_info : buf_info->pool_candidates) {
+        // Mark pool candidates that satisfy the size constraints.
+        if (IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+          pool_offset_candidates[pool_info] = 0;
+        }
+      }
+
+      for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+        auto conflict_buf_info = Downcast<BufferInfo>(conflict_buf_info_obj);
+        size_t next_offset = 0;
+        // We only look at already allocated BufferInfo in-terms of conflicts.
+        if (pool_allocations.count(conflict_buf_info)) {
+          auto pool_allocation = pool_allocations[conflict_buf_info];
+          next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+          next_offset =
+              round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+          // Checks whether the next offset in the same pool as the conflicting BufferInfo is valid.
+          if (IsValidPlacement(pool_allocation->pool_info, next_offset,
+                               buf_info->size_bytes->value)) {
+            // There could be multiple conflicting BufferInfo in the same pool.
+            // Thus, we need to make sure we pick the largest offset of them all.
+            if (next_offset > pool_offset_candidates[pool_allocation->pool_info]) {
+              pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+            }
+          } else {
+            pool_offset_candidates.erase(pool_allocation->pool_info);
+          }
+        }
+      }
+      auto selected_pool = SelectPlacementPool(buf_info->pool_candidates, pool_offset_candidates);
+      pool_allocations.Set(
+          buf_info, PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool])));
+    }
+    return pool_allocations;
+  }
+};
+
+/*!
+ * \brief This class implements Greedy by the size of BufferInfo
+ * greedy algorithm. Please refer to main documentation of the file
+ * for more details.
+ */
+class GreedySize : public GreedyBase {
+ public:
+  GreedySize() {}
+  Map<BufferInfo, PoolAllocation> PlanMemory(const Array<BufferInfo>& buffer_info_arr) {
+    std::vector<BufferInfo> buffer_info_vec;
+    Map<BufferInfo, PoolAllocation> pool_allocations;
+    for (const auto& buffer_info : buffer_info_arr) {
+      buffer_info_vec.push_back(std::move(buffer_info));
+    }
+    std::sort(buffer_info_vec.begin(), buffer_info_vec.end(),
+              [](const BufferInfo& a, const BufferInfo& b) {
+                if (a->size_bytes->value == b->size_bytes->value) {
+                  if (a->conflicts.size() == b->conflicts.size()) {
+                    auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                    auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                    return a_name_hash > b_name_hash;

Review comment:
       Well without this; the tests become unstable because of the presence of buffers of same size and conflicts. What would be your suggestion here for a test case ?

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -415,18 +454,30 @@ Map<BufferInfo, tir::Stmt> BufferInfoExtractor::operator()(const PrimFunc& main_
 
   // Traverse the liveness events using a open set to track what
   // is live while updating the conflicts through out the linear traversal
-  std::unordered_set<BufferInfo, ObjectPtrHash, ObjectPtrEqual> open_set;
+  //  std::unordered_set<BufferInfo, ObjectPtrHash, ObjectPtrEqual> open_set;

Review comment:
       Ack.

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -203,29 +214,40 @@ void BufferInfoExtractor::RecordAllocateNodeInfo(const AllocateNode* op) {
   auto size_bytes = CalculateExtentsSize(op);
   // We only statically memory plan only allocates with known
   // compile time sizes.
-  if (size_bytes.defined() &&
-      allocate_var_to_stmt_map_.find(op->buffer_var) == allocate_var_to_stmt_map_.end()) {
-    // By default, the core compiler is assumed to attach the a default pool to each allocate.
-    ICHECK(op->annotations.count(kPoolCandidatesAllocateAttr))
-        << "Every statically sized allocate node needs an pool candidate attribute";
-    auto pool_candidates = Downcast<Array<PoolInfo>>(op->annotations[kPoolCandidatesAllocateAttr]);
-
-    // TODO(@manupa-arm): improve the error when the responsible component for attaching a single
-    // pool is added
-    ICHECK(pool_candidates.size() > 0)
-        << "The core compiler should at least attach a single PoolInfo. If there were no "
-           "user-given arguments for memory pools, the default behaviour is a single size "
-           "un-restricted pool is assigned";
-    PrimFunc func = scope_stack_.top().func;
-    Optional<Target> tgt = func->GetAttr<Target>(tvm::attr::kTarget);
-    ICHECK(tgt) << "There should not be any PrimFuncs without a target attached by now";
-    auto workspace_alignment =
-        tgt.value()->GetAttr<Integer>("workspace-byte-alignment").value_or(16);
-    auto buffer_info = BufferInfo(GetUniqueBufferName(op->buffer_var->name_hint), size_bytes,
-                                  pool_candidates, workspace_alignment);
-    auto allocate = GetRef<Allocate>(op);
-    allocate_var_to_stmt_map_.Set(op->buffer_var, allocate);
-    buffer_info_map_.Set(buffer_info, allocate);
+  if (size_bytes.defined()) {

Review comment:
       https://github.com/apache/tvm/pull/9214/commits/26fffb0fbfac1558bef905dee1817ba2e5eeabdf.
   
   It is mentioned in "a commit" message. I ll capture this into the main message of the PR.
   I appreciate that it is truly not part of the greedy algorithm, however getting too small value after the greedy allocation is what actually made me find the issue. 
   
   Therefore, I hope updating the main PR commit message should help here.

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -415,18 +454,30 @@ Map<BufferInfo, tir::Stmt> BufferInfoExtractor::operator()(const PrimFunc& main_
 
   // Traverse the liveness events using a open set to track what
   // is live while updating the conflicts through out the linear traversal
-  std::unordered_set<BufferInfo, ObjectPtrHash, ObjectPtrEqual> open_set;
+  //  std::unordered_set<BufferInfo, ObjectPtrHash, ObjectPtrEqual> open_set;
+  std::unordered_map<BufferInfo, int, ObjectPtrHash, ObjectPtrEqual> open_set;
   for (const auto& le_event : le_events_timeline) {
     if (le_event.le_type == START) {
-      for (const auto& open_buffer_info : open_set) {
+      for (const auto& kv : open_set) {
+        BufferInfo open_buffer_info = kv.first;
         open_buffer_info->conflicts.push_back(le_event.buffer_info);
         if (le_event.buffer_info != open_buffer_info) {
           le_event.buffer_info->conflicts.push_back(open_buffer_info);
         }
       }
-      open_set.insert(le_event.buffer_info);
+      //      open_set.insert(le_event.buffer_info);
+      if (open_set.find(le_event.buffer_info) == open_set.end()) {
+        open_set[le_event.buffer_info] = 1;
+      } else {
+        open_set[le_event.buffer_info] += 1;
+      }
     } else {
-      open_set.erase(le_event.buffer_info);
+      if (open_set[le_event.buffer_info] == 1) {
+        open_set.erase(le_event.buffer_info);
+      } else {
+        open_set[le_event.buffer_info] -= 1;
+      }
+      //      open_set.erase(le_event.buffer_info);

Review comment:
       Thanks!

##########
File path: tests/python/unittest/test_tir_usmp_algo.py
##########
@@ -0,0 +1,492 @@
+# 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, script
+from tvm.script import tir as T
+from tvm.tir import stmt_functor
+from tvm.tir.usmp import utils as usmp_utils
+from tvm.target import Target
+
+
+def _replace_stmt_with_buf_var_names(buffer_info_map):
+    """helper to replace tir.allocates with buffer names"""
+    new_buffer_info_map = dict()
+    for k, v in buffer_info_map.items():
+        new_buffer_info_map[v.buffer_var.name] = k
+    return new_buffer_info_map
+
+
+def _verify_conflicts(main_buf_name, conflicting_buf_names, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    buf_info = buffer_info_map[main_buf_name]
+    for conflict in buf_info.conflicts:
+        assert conflict.name_hint in conflicting_buf_names
+
+
+def _get_allocates(primfunc):
+    """helper to extract all allocate nodes by name"""
+    allocates = dict()
+
+    def get_allocate(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            allocates[str(stmt.buffer_var.name)] = stmt
+
+    stmt_functor.post_order_visit(primfunc.body, get_allocate)
+    return allocates
+
+
+def assign_poolinfos_to_allocates_in_primfunc(primfunc, pool_infos):
+    """helper to assing poolinfos to allocate nodes in a tir.PrimFunc"""
+
+    def set_poolinfos(stmt):
+        if isinstance(stmt, tvm.tir.Allocate):
+            return tvm.tir.Allocate(
+                buffer_var=stmt.buffer_var,
+                dtype=stmt.dtype,
+                extents=stmt.extents,
+                condition=stmt.condition,
+                body=stmt.body,
+                annotations={tvm.tir.usmp.utils.CANDIDATE_MEMORY_POOL_ATTR: pool_infos},
+            )
+
+    return primfunc.with_body(stmt_functor.ir_transform(primfunc.body, None, set_poolinfos))
+
+
+def assign_poolinfos_to_allocates_in_irmodule(mod, pool_infos):
+    """helper to assing poolinfos to allocate nodes in a IRModule"""
+    ret = tvm.IRModule()
+    for global_var, basefunc in mod.functions.items():
+        if isinstance(basefunc, tvm.tir.PrimFunc):
+            ret[global_var] = assign_poolinfos_to_allocates_in_primfunc(basefunc, pool_infos)
+    return ret
+
+
+def _assign_targets_to_primfuncs_irmodule(mod, target):
+    """helper to assign target for PrimFunc in a IRModule"""
+    ret = tvm.IRModule()
+    for global_var, basefunc in mod.functions.items():
+        if isinstance(basefunc, tvm.tir.PrimFunc):
+            ret[global_var] = basefunc.with_attr("target", target)
+    return ret
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    assert max_workspace_size == size
+
+
+# fmt: off
+@tvm.script.ir_module
+class LinearStructure:
+    @T.prim_func
+    def tvmgen_default_fused_cast_subtract(placeholder_2: T.handle, placeholder_3: T.handle, T_subtract: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_cast_subtract", "tir.noalias": True})
+        placeholder_4 = T.match_buffer(placeholder_2, [1, 224, 224, 3], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        placeholder_5 = T.match_buffer(placeholder_3, [], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        T_subtract_1 = T.match_buffer(T_subtract, [1, 224, 224, 3], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        # body
+        for ax0_ax1_fused_1 in T.serial(0, 224):
+            for ax2_1, ax3_inner_1 in T.grid(224, 3):
+                T.store(T_subtract_1.data, (((ax0_ax1_fused_1*672) + (ax2_1*3)) + ax3_inner_1), (T.cast(T.load("uint8", placeholder_4.data, (((ax0_ax1_fused_1*672) + (ax2_1*3)) + ax3_inner_1)), "int16") - T.load("int16", placeholder_5.data, 0)), True)
+
+    @T.prim_func
+    def tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast(placeholder_62: T.handle, placeholder_63: T.handle, placeholder_64: T.handle, T_cast_20: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast", "tir.noalias": True})
+        placeholder_65 = T.match_buffer(placeholder_62, [1, 224, 224, 3], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        placeholder_66 = T.match_buffer(placeholder_63, [7, 7, 3, 64], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        placeholder_67 = T.match_buffer(placeholder_64, [1, 1, 1, 64], dtype="int32", elem_offset=0, align=128, offset_factor=1)
+        T_cast_21 = T.match_buffer(T_cast_20, [1, 112, 112, 64], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        # body
+        PaddedInput_7 = T.allocate([157323], "int16", "global")
+        for i0_i1_fused_7 in T.serial(0, 229):
+            for i2_7, i3_7 in T.grid(229, 3):
+                T.store(PaddedInput_7, (((i0_i1_fused_7*687) + (i2_7*3)) + i3_7), T.if_then_else(((((2 <= i0_i1_fused_7) and (i0_i1_fused_7 < 226)) and (2 <= i2_7)) and (i2_7 < 226)), T.load("int16", placeholder_65.data, ((((i0_i1_fused_7*672) + (i2_7*3)) + i3_7) - 1350)), T.int16(0), dtype="int16"), True)
+        for ax0_ax1_fused_ax2_fused_7 in T.serial(0, 12544):
+            Conv2dOutput_7 = T.allocate([64], "int32", "global")
+            for ff_3 in T.serial(0, 64):
+                T.store(Conv2dOutput_7, ff_3, 0, True)
+                for ry_2, rx_2, rc_7 in T.grid(7, 7, 3):
+                    T.store(Conv2dOutput_7, ff_3, (T.load("int32", Conv2dOutput_7, ff_3) + (T.cast(T.load("int16", PaddedInput_7, (((((T.floordiv(ax0_ax1_fused_ax2_fused_7, 112)*1374) + (ry_2*687)) + (T.floormod(ax0_ax1_fused_ax2_fused_7, 112)*6)) + (rx_2*3)) + rc_7)), "int32")*T.cast(T.load("int16", placeholder_66.data, ((((ry_2*1344) + (rx_2*192)) + (rc_7*64)) + ff_3)), "int32"))), True)
+            for ax3_inner_7 in T.serial(0, 64):
+                T.store(T_cast_21.data, ((ax0_ax1_fused_ax2_fused_7*64) + ax3_inner_7), T.cast(T.max(T.min(T.q_multiply_shift((T.load("int32", Conv2dOutput_7, ax3_inner_7) + T.load("int32", placeholder_67.data, ax3_inner_7)), 1939887962, 31, -9, dtype="int32"), 255), 0), "uint8"), True)
+
+    @T.prim_func
+    def tvmgen_default_fused_nn_max_pool2d_cast(placeholder_28: T.handle, T_cast_6: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_fused_nn_max_pool2d_cast", "tir.noalias": True})
+        placeholder_29 = T.match_buffer(placeholder_28, [1, 112, 112, 64], dtype="uint8", elem_offset=0, align=128, offset_factor=1)
+        T_cast_7 = T.match_buffer(T_cast_6, [1, 56, 56, 64], dtype="int16", elem_offset=0, align=128, offset_factor=1)
+        # body
+        tensor_2 = T.allocate([200704], "uint8", "global")
+        for ax0_ax1_fused_4 in T.serial(0, 56):
+            for ax2_4 in T.serial(0, 56):
+                for ax3_init in T.serial(0, 64):
+                    T.store(tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_init), T.uint8(0), True)
+                for rv0_rv1_fused_1, ax3_2 in T.grid(9, 64):
+                    T.store(tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_2), T.max(T.load("uint8", tensor_2, (((ax0_ax1_fused_4*3584) + (ax2_4*64)) + ax3_2)), T.if_then_else(((((ax0_ax1_fused_4*2) + T.floordiv(rv0_rv1_fused_1, 3)) < 112) and (((ax2_4*2) + T.floormod(rv0_rv1_fused_1, 3)) < 112)), T.load("uint8", placeholder_29.data, (((((ax0_ax1_fused_4*14336) + (T.floordiv(rv0_rv1_fused_1, 3)*7168)) + (ax2_4*128)) + (T.floormod(rv0_rv1_fused_1, 3)*64)) + ax3_2)), T.uint8(0), dtype="uint8")), True)
+        for ax0_ax1_fused_5 in T.serial(0, 56):
+            for ax2_5, ax3_3 in T.grid(56, 64):
+                T.store(T_cast_7.data, (((ax0_ax1_fused_5*3584) + (ax2_5*64)) + ax3_3), T.cast(T.load("uint8", tensor_2, (((ax0_ax1_fused_5*3584) + (ax2_5*64)) + ax3_3)), "int16"), True)
+
+    @T.prim_func
+    def run_model(input: T.handle, output: T.handle) -> None:
+        # function attr dict
+        T.func_attr({"global_symbol": "tvmgen_default_run_model", "runner_function": True})
+        # body
+        T.attr("default", "device_id", 0)
+        T.attr("default", "device_type", 1)
+        sid_9 = T.allocate([301056], "int8", "global")
+        sid_8 = T.allocate([802816], "int8", "global")
+        T.evaluate(T.call_extern("tvmgen_default_fused_cast_subtract", input, T.lookup_param("p0", dtype="handle"), sid_9, dtype="int32"))
+        T.evaluate(T.call_extern("tvmgen_default_fused_nn_conv2d_add_fixed_point_multiply_clip_cast", sid_9, T.lookup_param("p1", dtype="handle"), T.lookup_param("p2", dtype="handle"), sid_8, dtype="int32"))
+        T.evaluate(T.call_extern("tvmgen_default_fused_nn_max_pool2d_cast", sid_8, output, dtype="int32"))
+    __tvm_meta__ = None
+# fmt: on
+
+
+def print_conflicts(buffer_info_map):
+    """_verify_conflicts("sid_8", ["Conv2dOutput_7", "tensor_2"], buffer_info_map)"""
+
+    for buffer_info_name, buf_info in buffer_info_map.items():
+        conflict_str = "["
+        for conflict in buf_info.conflicts:
+            conflict_str += f'"{conflict.name_hint}", '
+        conflict_str += "]"
+        print(f'_verify_conflicts("{buffer_info_name}", {conflict_str}, buffer_info_map_names)')
+
+
+@pytest.mark.parametrize(
+    ["algorithm", "fast_memory_size", "slow_memory_size"],
+    [("greedy_by_size", 200704, 1418528), ("greedy_by_conflicts", 200704, 1418528)],
+)
+def test_linear(algorithm, fast_memory_size, slow_memory_size):
+    target = Target("c")
+    fast_memory_pool = usmp_utils.PoolInfo(
+        pool_name="fast_memory",
+        target_access={target: usmp_utils.PoolInfo.READ_WRITE_ACCESS},
+        size_hint_bytes=200704,
+    )
+    slow_memory_pool = usmp_utils.PoolInfo(
+        pool_name="slow_memory", target_access={target: usmp_utils.PoolInfo.READ_WRITE_ACCESS}
+    )
+    tir_mod = LinearStructure
+    tir_mod = _assign_targets_to_primfuncs_irmodule(tir_mod, target)
+    tir_mod = assign_poolinfos_to_allocates_in_irmodule(
+        tir_mod, [fast_memory_pool, slow_memory_pool]
+    )
+    main_func = tir_mod["run_model"]
+    buffer_info_map = tvm.tir.usmp.analysis.extract_buffer_info(main_func, tir_mod)
+
+    fcreate_array_bi = tvm.get_global_func("tir.usmp.CreateArrayBufferInfo")
+    buffer_info_arr = fcreate_array_bi(buffer_info_map)
+    fusmp_algo_greedy_by_size = tvm.get_global_func(f"tir.usmp.algo.{algorithm}")
+    buffer_pool_allocations = fusmp_algo_greedy_by_size(buffer_info_arr)
+
+    buffer_info_map_names = dict()
+    for buf_info in buffer_info_arr:
+        buffer_info_map_names[buf_info.name_hint] = buf_info
+
+    # check conflicts
+    _verify_conflicts("PaddedInput_7", ["sid_9", "sid_8", "Conv2dOutput_7"], buffer_info_map_names)
+    _verify_conflicts("tensor_2", ["sid_8"], buffer_info_map_names)
+    _verify_conflicts("sid_9", ["PaddedInput_7"], buffer_info_map_names)
+    _verify_conflicts(
+        "sid_8", ["PaddedInput_7", "Conv2dOutput_7", "tensor_2"], buffer_info_map_names
+    )
+    _verify_conflicts("Conv2dOutput_7", ["sid_8", "PaddedInput_7"], buffer_info_map_names)
+
+    _check_max_workspace_size(buffer_pool_allocations, slow_memory_pool, slow_memory_size)
+    _check_max_workspace_size(buffer_pool_allocations, fast_memory_pool, fast_memory_size)
+
+
+# fmt: off
+@tvm.script.ir_module
+class ResnetStructure:

Review comment:
       It has to be a fan-out with varied intermediary buffers as well and getting a snippet from the ResNet model was reasonably easy and seemed reasonable to me. 
   Again, if you have suggestion, I would be happy to consider.

##########
File path: src/tir/usmp/algo/greedy.cc
##########
@@ -0,0 +1,235 @@
+/*
+ * 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 tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief This source contains greedy algorithms for planning
+ * memory for USMP. There are two algorithms present here :
+ * 1) greedy_by_size and 2) greedy_by_conflicts.
+ *
+ * greedy_by_size : this algorithm prioritizes placing the
+ * largest size buffer to the given pools. The BufferInfo objects
+ * are sorted based on the size and placed on each pool adhering
+ * to size_hint constraint.
+ *
+ * greedy_by_conflicts : this algorithm prioritizes placing the
+ * the most liveness conflicted buffer to the given pools. The
+ * BufferInfo objects are sorted based on the number of conflicts
+ * and placed on each pool adhering to size_hint constraint.
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+
+/*!
+ * \brief This is the base class for Greedy Algorithms where the sorting
+ * is specialized in the extended classes based on the greedy criteria.
+ */
+class GreedyBase {
+ public:
+  GreedyBase() {}
+  /*!
+   * \brief This function should be implemented by the extended classes to sort the BufferInfo
+   * objects based on a criteria and then calling PostSortAllocation.
+   */
+  virtual Map<BufferInfo, PoolAllocation> PlanMemory(const Array<BufferInfo>& buffer_info_arr) = 0;
+
+ protected:
+  /*!
+   * \brief Rounds up the offset to satisfy the alignement requirement
+   */
+  size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                    const int& byte_alignment) {
+    return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+  }
+
+  /*!
+   * \brief A helper function check whether a offset is valid given the constraints
+   */
+  bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                        const size_t& size_bytes) {
+    if (candidate_pool->size_hint_bytes == -1) {
+      // this means pool is not bounded
+      return true;
+    }
+    auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+    auto max_address = next_offset + size_bytes;
+    if (max_address <= pool_size) {
+      return true;
+    }
+    return false;
+  }
+
+  /*!
+   * \brief Selects a pool for placement in the given set of ordered pool candidates
+   */
+  PoolInfo SelectPlacementPool(
+      const Array<PoolInfo>& pool_candidates,
+      const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+    // Here the pool candidates are ordered when it is consumed by the algorithm.
+    // This could be from order the user has specified. However, schedulers are
+    // welcome to change the order for performance reasons.
+    for (const auto& pool_info : pool_candidates) {
+      if (pool_offsets.count(pool_info)) {
+        return pool_info;
+      }
+    }
+    ICHECK(false) << "TVM USMP Internal Error: no candidate have been selected!";
+    return PoolInfo();
+  }
+
+  /*!
+   * \brief This is the base allocation function that works on sorted BufferInfo objects based
+   * on the greedy heuristic. The sorting algorithm has to be called before calling this.
+   */
+  Map<BufferInfo, PoolAllocation> PostSortAllocation(
+      const std::vector<BufferInfo>& buffer_info_vec) {
+    Map<BufferInfo, PoolAllocation> pool_allocations;
+    for (const auto& buf_info : buffer_info_vec) {
+      std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+      for (const auto& pool_info : buf_info->pool_candidates) {
+        // Mark pool candidates that satisfy the size constraints.
+        if (IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+          pool_offset_candidates[pool_info] = 0;
+        }
+      }
+
+      for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+        auto conflict_buf_info = Downcast<BufferInfo>(conflict_buf_info_obj);
+        size_t next_offset = 0;

Review comment:
       I favored perf. portability as I dont expect it to overflow.




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