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/10/06 18:36:20 UTC

[GitHub] [tvm] manupa-arm opened a new pull request #9214: [TIR][USMP] Greedy memory planning algorithm

manupa-arm opened a new pull request #9214:
URL: https://github.com/apache/tvm/pull/9214


   This commit implements a greedy by size memory planning algorithm using proposed USMP design (See the [RFC](https://github.com/apache/tvm-rfcs/blob/main/rfcs/0009_Unified_Static_Memory_Planning.md) for more details).
   
   * Added a test case of linear sequence of operators with two pools
   * Added a test case with residual structures
   
   This is blocked on #9168 and #8468 being merged. Thus this is based on top of those PRs.
   
   


-- 
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] mbaret commented on pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret commented on pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#issuecomment-984466990


   This is now merged, thanks @manupa-arm, @mbs-octoml!


-- 
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] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759229233



##########
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:
       > That is, unless, those integration tests will be extended for every new algorithm that's added (which is plausible).
   
   Yes, we will be doing this as that would make the most sense as that is the only sensible way to make sure the number workspace size decided by the algorithm, actually turns out to be something that runs.
   
   > Regarding these specific tests, I think they would be a lot more readable using small synthetic tests rather than network extracts. The linear case has a large amount of complexity that doesn't contribute to the test if topology is the only impactful change.
   
   I think, in terms of unit testing this algorithm, I ll create BufferInfo object based tests rather than TIR ones -- truly thats the input to the algorithm.




-- 
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] mbaret commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759213453



##########
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:
       Well ResNet has padded convolutions and heavily fused kernels in it with weird names. This makes it quite hard to read - for me at least - and I don't think that complexity enhances a test of fan-out behaviour. It would seem that to test fan-out all we strictly need is a couple of simple elementwise operations that read from the same tensor. I think this test still has merit (to test a more complex 'real' case), but probably needs naming as such (i.e. test_resnet_subgraph).




-- 
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] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759229233



##########
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:
       > That is, unless, those integration tests will be extended for every new algorithm that's added (which is plausible).
   
   Yes, we will be doing this as that would make the most sense as that is the only sensible way to make sure the number workspace size decided by the algorithm, actually turns out to be something that runs.
   
   > Regarding these specific tests, I think they would be a lot more readable using small synthetic tests rather than network extracts. The linear case has a large amount of complexity that doesn't contribute to the test if topology is the only impactful change.
   
   I think, in terms of unit testing this algorithm, I ll create BufferInfo object based tests rather than TIR ones -- truly thats in the input to the algorithm.




-- 
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] manupa-arm commented on pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#issuecomment-982515040


   @mbaret Could you take look at this PR when you have some time ?


-- 
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] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759231998



##########
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:
       I ll add few BufferInfo based conflict graphs additionally to renaming this test that would result from a simpler fan out 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] mbaret commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759217918



##########
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:
       I think it becomes a bit harder to argue for testing coverage in integration tests when this algorithm component is 'swappable'. That is, unless, those integration tests will be extended for every new algorithm that's added (which is plausible).
   
   Regarding these specific tests, I think they would be a lot more readable using small synthetic tests rather than network extracts. The linear case has a large amount of complexity that doesn't contribute to the test if topology is the only impactful 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] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759619419



##########
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:
       Done

##########
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:
       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] mbaret merged pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret merged pull request #9214:
URL: https://github.com/apache/tvm/pull/9214


   


-- 
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] mbs-octoml commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbs-octoml commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r755593603



##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(const Array<BufferInfo>& buffer_info_arr) {
+  std::vector<BufferInfo> buffer_info_vec;

Review comment:
       buffer_info_vec(buffer_info_arr.begin(), buffer_info_arr.end()) is fine I believe.
   shame about all this Map/unoredered_map Array/vector confusion.

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(const Array<BufferInfo>& buffer_info_arr) {
+  std::vector<BufferInfo> buffer_info_vec;
+  Map<BufferInfo, PoolAllocation> pool_allocations;

Review comment:
       move down to main loop

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(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;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes > b->size_bytes;
+            });
+
+  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) {
+      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;
+      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);
+        if (IsValidPlacement(pool_allocation->pool_info, next_offset,
+                             buf_info->size_bytes->value)) {
+          if (next_offset > pool_offset_candidates[pool_allocation->pool_info]) {
+            pool_offset_candidates[pool_allocation->pool_info] = next_offset;

Review comment:
       this if in particular confuses me

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  for (const auto& pool_info : pool_candidates) {

Review comment:
       kinda a heuristic too, call out?

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(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;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes > b->size_bytes;
+            });
+
+  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) {

Review comment:
       only consider pools large enough
   explain you don't need to track assigned buffers here (there zero constant surprised me) since everything is captured by the conflict buffers. 

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(const Array<BufferInfo>& buffer_info_arr) {

Review comment:
       comment

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm

Review comment:
       A few sentences describing the algorithm would be nice.

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(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) {

Review comment:
       can you consistently use the ->value form.
   explain heuristic -- large-to-small,  more-conflicted-to-less-conflicted.

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static 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;
+}
+
+static 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;
+}
+
+static PoolInfo SelectPlacementPool(
+    const Array<PoolInfo>& pool_candidates,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  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();
+}
+
+Map<BufferInfo, PoolAllocation> GreedyBySize(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;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes > b->size_bytes;
+            });
+
+  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) {
+      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);

Review comment:
       comments under each if will help here.
   basically shift the candidate allocation to avoid already committed assignments for conflicting buffers.

##########
File path: src/tir/usmp/algo/greedy_by_size.cc
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 Implement greedy by size memory planning algorithm
+ */
+
+#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 {
+
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,

Review comment:
       nit: suggest more idiomatic anonymous namespace instead of the static convention.




-- 
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] manupa-arm commented on pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#issuecomment-956528002


   a friendly ping @mbs-octoml ! this depends on #8468 


-- 
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] manupa-arm commented on pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#issuecomment-981930276


   Thanks! @mbs-octoml .
   
   I finally got around to address the comments and did some cleanup.


-- 
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] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759202971



##########
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 relavant to me. 
   Again, if you have suggestion, I would be happy to consider.




-- 
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] manupa-arm commented on pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#issuecomment-983744918


   Ah the CI decided to run itself again, thus I've changed the error message.


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

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

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



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

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759229816



##########
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:
       As suggested above, I ll create a BufferInfo based test for this one.

##########
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:
       As suggested above, I ll create a BufferInfo based test for this one too.




-- 
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] mbaret commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r760238128



##########
File path: src/tir/usmp/algo/greedy.cc
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.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 BufferInfo& buf_info,
+      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 : buf_info->pool_candidates) {
+      if (pool_offsets.count(pool_info)) {
+        return pool_info;
+      }
+    }
+    CHECK(false) << "TVM USMP Error: no candidate have been selected for " << buf_info;

Review comment:
       I think this needs to be more obvious because it'll be directly user-facing. Probably it needs to indicate that TVM has exceeded the pool memory limits and suggest to increase them if you want the compilation to pass. Let's not block the whole patch on this though, happy to take it in a follow-up.




-- 
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] mbaret commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759221195



##########
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:
       I'd suggest a small case which explicitly has buffers of the same size and conflicts and confirms they are prioritized according to name hash. It's not immediately obvious from reading the TIR for test_linear or test_fanout that they contain such cases (not saying they necessarily don't), so I think a standalone test is preferable.




-- 
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] manupa-arm commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759174589



##########
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:
       remove

##########
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:
       remove

##########
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:
       When would this check be reached (if ever)? It seems like it may be hit if every pool is full, in which case I think the reported error should be quite explicit as it's something the user could reasonably run into. We should also probably test it as I don't think it would be a 'true assert'.

##########
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've had historic queries around size_t vs. uint64_t in these situations (where sizeof/indexing isn't really being used). I don't have a strong view either way, but wondering if you've got a particular reason to favour size_t here?

##########
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:
       should these also be 'private' i.e. _assign?

##########
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:
       doesn't match current location

##########
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:
       remove

##########
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:
       I'm not sure this edge-case behaviour is explicitly tested.

##########
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:
       I think these changes need rationalizing in the commit message as I can't immediately see how they relate to the greedy algorithm.

##########
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:
       As a general comment on these tests, I'd normally favour a larger number of smaller cases. It's quite hard to parse these complex TIR modules and determine the expected behaviour of the test. Additionally, it's ambiguous what coverage we have other than it works in a particular linear and fan-out case.

##########
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:
       The naming here (and fusmp_algo_greedy_by_size) is a bit confusing because 'algorithm' could be by conflict I think.

##########
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:
       If the behaviour we're testing is specifically fan-out, I think this can be done with a simpler case than ResNet.




-- 
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] mbaret commented on a change in pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#discussion_r759219077



##########
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:
       Yeah that'd be fine.




-- 
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] manupa-arm commented on pull request #9214: [TIR][USMP] Greedy memory planning algorithm

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #9214:
URL: https://github.com/apache/tvm/pull/9214#issuecomment-956528002


   a friendly ping @mbs-octoml ! this depends on #8468 


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