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 2020/03/24 20:44:32 UTC

[GitHub] [incubator-tvm] jroesch opened a new pull request #5144: [Relay][VM] Memory planner (part 1)

jroesch opened a new pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144
 
 
   This PR adds a new pass to the memory scheduling phase of the VM. After we have made all memory allocations explicit, we now analyze for basic block style regions and allocate a single piece of storage for each. 
   
   The next PR will focus on how to overlap these to do memory compression, i.e reduce the amount of live memory. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406983327
 
 

 ##########
 File path: python/tvm/relay/def_use.py
 ##########
 @@ -0,0 +1,42 @@
+from . import expr
+from .expr_functor import ExprVisitor
+from .analysis import free_vars
+
+import attr
+from typing import List, Dict
+
+@attr.s(auto_attribs=True)
+class DefUse:
+    defn: expr.Var
+    uses: List[expr.Expr]
+
+class DefUseAnalysis(ExprVisitor):
+    def __init__(self):
+        super().__init__()
+        self.results: Dict[expr.Var, DefUse] = {}
+
+    def visit_function(self, func):
+        for param in func.params:
+            self.results[param] = DefUse(param, [])
+        super().visit_function(func)
+
+    def visit_let(self, let):
+        while isinstance(let, expr.Let):
+            du = DefUse(let.var, [])
+            self.results[let.var] = du
+            # Find all variables used in RHS.
+            self.visit(let.value)
+            used_vars = free_vars(let.value)
+            for uvar in used_vars:
+                self.results[uvar].uses.append(let.value)
+            let = let.body
+
+        # Find all variables used in body.
+        used_vars = free_vars(let)
 
 Review comment:
   let here is actually the body

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r397530214
 
 

 ##########
 File path: python/tvm/relay/transform/memory_plan.py
 ##########
 @@ -0,0 +1,189 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+import attr
+import numpy as np
+from typing import Optional, Dict
+
+from ..expr_functor import ExprMutator
+from ..scope_builder import ScopeBuilder
+from .. import op, ty, expr
+from ... import DataType, register_func, IRModule
+from .. import analysis
+from . import FoldConstant, InferType, function_pass
+from ..backend import compile_engine
+
+def is_primitive(call):
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
+           hasattr(call.op.attrs, 'Primitive') and int(call.op.attrs.Primitive) == 1
+
+@attr.s(auto_attribs=True)
+class Region:
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    offsets: Dict[expr.Var, expr.Expr] = {}
+
+    def grow(self, old_storage: expr.Var, size: expr.Expr, alignment: expr.Expr, dtype: str) -> None:
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert analysis.alpha_equal(self.alignment, alignment), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        # Record the offset at which we allocate the storage.
+        self.offsets[old_storage] = self.size
+
+        self.size = self.size + size
+
+    def to_expr(self) -> expr.Expr:
+        return op.memory.alloc_storage(self.size, self.alignment, self.dtype)
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+def mk_let(bindings, body):
 
 Review comment:
   letlist is everywhere. can you put this in some common file?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r410419288
 
 

 ##########
 File path: python/tvm/relay/transform/memory_alloc.py
 ##########
 @@ -26,8 +26,13 @@
 from ... import DataType, register_func
 from .. import ty, expr
 from ..backend import compile_engine
+from ..op.memory import alloc_storage, alloc_tensor
 
 
+def alloc_tensor(storage, shape, dtype='float32', assert_shape=None):
+    offset = expr.const(0, dtype="int64")
 
 Review comment:
   We initially allocate 0, then we later change the offset to be non-zero. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jroesch commented on issue #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
jroesch commented on issue #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#issuecomment-603496275
 
 
   cc @zhiics @yidawang @icemelon9 @MarisaKirisame 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406985977
 
 

 ##########
 File path: python/tvm/relay/transform/memory_plan.py
 ##########
 @@ -0,0 +1,189 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+import attr
+import numpy as np
+from typing import Optional, Dict
+
+from ..expr_functor import ExprMutator
+from ..scope_builder import ScopeBuilder
+from .. import op, ty, expr
+from ... import DataType, register_func, IRModule
+from .. import analysis
+from . import FoldConstant, InferType, function_pass
+from ..backend import compile_engine
+
+def is_primitive(call):
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
+           hasattr(call.op.attrs, 'Primitive') and int(call.op.attrs.Primitive) == 1
+
+@attr.s(auto_attribs=True)
+class Region:
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    offsets: Dict[expr.Var, expr.Expr] = {}
+
+    def grow(self, old_storage: expr.Var, size: expr.Expr, alignment: expr.Expr, dtype: str) -> None:
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert analysis.alpha_equal(self.alignment, alignment), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        # Record the offset at which we allocate the storage.
+        self.offsets[old_storage] = self.size
+
+        self.size = self.size + size
+
+    def to_expr(self) -> expr.Expr:
+        return op.memory.alloc_storage(self.size, self.alignment, self.dtype)
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        body = expr.Let(var, value, body)
+    return body
+
+class StorageCoalesce(ExprMutator):
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self):
+        zero = expr.const(0, dtype="int64")
+        region_var = expr.var(f"region{len(self.regions)}")
+        region = Region(region_var, zero, None, None)
+        self.regions.append(region)
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        region = self.regions.pop()
+        storage_expr = region.to_expr()
+        assert storage_expr, "can not be None"
+        return expr.Let(region.var, storage_expr, body)
+
+    def current_region(self) -> Region:
+        return self.regions[-1]
+
+    def visit_function(self, function):
+        if function.attrs and int(function.attrs.Primitive) == 1:
+            return super().visit_function(function)
 
 Review comment:
   QQ, do we need to step into primitive function? I thought we should probably just return it, right?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406977664
 
 

 ##########
 File path: python/tvm/relay/transform/memory_plan.py
 ##########
 @@ -0,0 +1,189 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+import attr
+import numpy as np
+from typing import Optional, Dict
+
+from ..expr_functor import ExprMutator
+from ..scope_builder import ScopeBuilder
+from .. import op, ty, expr
+from ... import DataType, register_func, IRModule
+from .. import analysis
+from . import FoldConstant, InferType, function_pass
+from ..backend import compile_engine
+
+def is_primitive(call):
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
+           hasattr(call.op.attrs, 'Primitive') and int(call.op.attrs.Primitive) == 1
+
+@attr.s(auto_attribs=True)
+class Region:
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    offsets: Dict[expr.Var, expr.Expr] = {}
+
+    def grow(self, old_storage: expr.Var, size: expr.Expr, alignment: expr.Expr, dtype: str) -> None:
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert analysis.alpha_equal(self.alignment, alignment), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        # Record the offset at which we allocate the storage.
+        self.offsets[old_storage] = self.size
+
+        self.size = self.size + size
+
+    def to_expr(self) -> expr.Expr:
+        return op.memory.alloc_storage(self.size, self.alignment, self.dtype)
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        body = expr.Let(var, value, body)
+    return body
+
+class StorageCoalesce(ExprMutator):
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self):
+        zero = expr.const(0, dtype="int64")
+        region_var = expr.var(f"region{len(self.regions)}")
+        region = Region(region_var, zero, None, None)
+        self.regions.append(region)
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        region = self.regions.pop()
+        storage_expr = region.to_expr()
+        assert storage_expr, "can not be None"
+        return expr.Let(region.var, storage_expr, body)
+
+    def current_region(self) -> Region:
+        return self.regions[-1]
+
+    def visit_function(self, function):
+        if function.attrs and int(function.attrs.Primitive) == 1:
+            return super().visit_function(function)
+        else:
+            self.enter_scope()
+            body = self.visit(function.body)
+            body = self.exit_scope(body)
+            return expr.Function(
+                function.params,
+                body,
+                function.ret_type,
+                function.type_params,
+                function.attrs)
+
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+    def visit_let(self, let):
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get("memory.alloc_storage"):
+                return self.process_alloc_storage(lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get("memory.alloc_tensor"):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        return iterative_let(let, _each_binding, mk_let)
+
+    def process_alloc_storage(self, lhs, call):
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        region = self.current_region()
+        region.grow(lhs, size, alignment, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        region = self.current_region()
+        storage, old_offset, shape = call.args
+        offset = region.offsets[storage]
+        assert old_offset.data.asnumpy().item() == 0, "no offsets should yet be allocated"
+        return lhs, expr.Call(call.op, [region.var, offset, shape], call.attrs, call.type_args)
+
+
+class MemoryPlanPass(ExprMutator):
+    """A pass for coalescing allocations made by the Relay VM."""
+    # def visit_let(self, let):
+    #     import pdb; pdb.set_trace()
+    # pass restore after rebase
 
 Review comment:
   remove the comments

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406979426
 
 

 ##########
 File path: include/tvm/relay/attrs/memory.h
 ##########
 @@ -31,6 +31,20 @@
 namespace tvm {
 namespace relay {
 
+/*!
+ * \brief Options for allocating storage.
+ */
+struct AllocStorageAttrs : public tvm::AttrsNode<AllocStorageAttrs> {
+  DataType dtype;
 
 Review comment:
   Why do we need the dtype attr for alloc storage?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406984489
 
 

 ##########
 File path: python/tvm/relay/transform/memory_plan.py
 ##########
 @@ -0,0 +1,189 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+import attr
+import numpy as np
+from typing import Optional, Dict
+
+from ..expr_functor import ExprMutator
+from ..scope_builder import ScopeBuilder
+from .. import op, ty, expr
+from ... import DataType, register_func, IRModule
+from .. import analysis
+from . import FoldConstant, InferType, function_pass
+from ..backend import compile_engine
+
+def is_primitive(call):
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
+           hasattr(call.op.attrs, 'Primitive') and int(call.op.attrs.Primitive) == 1
+
+@attr.s(auto_attribs=True)
+class Region:
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    offsets: Dict[expr.Var, expr.Expr] = {}
+
+    def grow(self, old_storage: expr.Var, size: expr.Expr, alignment: expr.Expr, dtype: str) -> None:
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert analysis.alpha_equal(self.alignment, alignment), "must have matching alignments in a region"
 
 Review comment:
   alpha_equal is now replaced by `tvm.ir.structural_equal` now

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#issuecomment-612260917
 
 
   @zhiics could you take a look at this pass?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406983887
 
 

 ##########
 File path: python/tvm/relay/transform/memory_alloc.py
 ##########
 @@ -26,8 +26,13 @@
 from ... import DataType, register_func
 from .. import ty, expr
 from ..backend import compile_engine
+from ..op.memory import alloc_storage, alloc_tensor
 
 
+def alloc_tensor(storage, shape, dtype='float32', assert_shape=None):
+    offset = expr.const(0, dtype="int64")
 
 Review comment:
   so that offset is actually always zero? then we can just directly use 0 when allocating tensor?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406986484
 
 

 ##########
 File path: python/tvm/relay/transform/memory_plan.py
 ##########
 @@ -0,0 +1,189 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+import attr
+import numpy as np
+from typing import Optional, Dict
+
+from ..expr_functor import ExprMutator
+from ..scope_builder import ScopeBuilder
+from .. import op, ty, expr
+from ... import DataType, register_func, IRModule
+from .. import analysis
+from . import FoldConstant, InferType, function_pass
+from ..backend import compile_engine
+
+def is_primitive(call):
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
+           hasattr(call.op.attrs, 'Primitive') and int(call.op.attrs.Primitive) == 1
+
+@attr.s(auto_attribs=True)
+class Region:
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    offsets: Dict[expr.Var, expr.Expr] = {}
+
+    def grow(self, old_storage: expr.Var, size: expr.Expr, alignment: expr.Expr, dtype: str) -> None:
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert analysis.alpha_equal(self.alignment, alignment), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        # Record the offset at which we allocate the storage.
+        self.offsets[old_storage] = self.size
+
+        self.size = self.size + size
+
+    def to_expr(self) -> expr.Expr:
+        return op.memory.alloc_storage(self.size, self.alignment, self.dtype)
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        body = expr.Let(var, value, body)
+    return body
+
+class StorageCoalesce(ExprMutator):
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self):
+        zero = expr.const(0, dtype="int64")
+        region_var = expr.var(f"region{len(self.regions)}")
+        region = Region(region_var, zero, None, None)
+        self.regions.append(region)
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        region = self.regions.pop()
+        storage_expr = region.to_expr()
+        assert storage_expr, "can not be None"
+        return expr.Let(region.var, storage_expr, body)
+
+    def current_region(self) -> Region:
+        return self.regions[-1]
+
+    def visit_function(self, function):
+        if function.attrs and int(function.attrs.Primitive) == 1:
+            return super().visit_function(function)
+        else:
+            self.enter_scope()
+            body = self.visit(function.body)
+            body = self.exit_scope(body)
+            return expr.Function(
+                function.params,
+                body,
+                function.ret_type,
+                function.type_params,
+                function.attrs)
+
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+    def visit_let(self, let):
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get("memory.alloc_storage"):
+                return self.process_alloc_storage(lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get("memory.alloc_tensor"):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        return iterative_let(let, _each_binding, mk_let)
+
+    def process_alloc_storage(self, lhs, call):
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        region = self.current_region()
+        region.grow(lhs, size, alignment, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        region = self.current_region()
+        storage, old_offset, shape = call.args
+        offset = region.offsets[storage]
+        assert old_offset.data.asnumpy().item() == 0, "no offsets should yet be allocated"
+        return lhs, expr.Call(call.op, [region.var, offset, shape], call.attrs, call.type_args)
+
+
+class MemoryPlanPass(ExprMutator):
+    """A pass for coalescing allocations made by the Relay VM."""
+    # def visit_let(self, let):
+    #     import pdb; pdb.set_trace()
+    # pass restore after rebase
+
+
+def eval_const(mod, func):
+    mod["tmp"] = func
+    mod = FoldConstant()(mod)
+    return mod["tmp"]
+
+def infer_type(mod, func):
 
 Review comment:
   We only need to keep `eval_const`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r410418958
 
 

 ##########
 File path: python/tvm/relay/def_use.py
 ##########
 @@ -0,0 +1,42 @@
+from . import expr
+from .expr_functor import ExprVisitor
+from .analysis import free_vars
+
+import attr
+from typing import List, Dict
+
+@attr.s(auto_attribs=True)
+class DefUse:
+    defn: expr.Var
+    uses: List[expr.Expr]
+
+class DefUseAnalysis(ExprVisitor):
 
 Review comment:
   This was in prep for the next piece I can remove from this PR. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406978538
 
 

 ##########
 File path: src/runtime/vm/vm.cc
 ##########
 @@ -535,6 +544,7 @@ void InstructionPrint(std::ostream& os, const Instruction& instr) {
     case Opcode::AllocTensorReg: {
       os << "alloc_tensor_reg $" << instr.dst << " $"
          << instr.alloc_tensor_reg.storage << " $"
+         << instr.alloc_tensor_reg.offset << " $"
 
 Review comment:
   ```suggestion
            << instr.alloc_tensor_reg.offset << " "
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406989002
 
 

 ##########
 File path: python/tvm/relay/def_use.py
 ##########
 @@ -0,0 +1,42 @@
+from . import expr
+from .expr_functor import ExprVisitor
+from .analysis import free_vars
+
+import attr
+from typing import List, Dict
+
+@attr.s(auto_attribs=True)
+class DefUse:
+    defn: expr.Var
+    uses: List[expr.Expr]
+
+class DefUseAnalysis(ExprVisitor):
+    def __init__(self):
+        super().__init__()
+        self.results: Dict[expr.Var, DefUse] = {}
+
+    def visit_function(self, func):
+        for param in func.params:
+            self.results[param] = DefUse(param, [])
+        super().visit_function(func)
+
+    def visit_let(self, let):
+        while isinstance(let, expr.Let):
+            du = DefUse(let.var, [])
+            self.results[let.var] = du
+            # Find all variables used in RHS.
+            self.visit(let.value)
+            used_vars = free_vars(let.value)
+            for uvar in used_vars:
+                self.results[uvar].uses.append(let.value)
+            let = let.body
+
+        # Find all variables used in body.
+        used_vars = free_vars(let)
 
 Review comment:
   i see.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406963966
 
 

 ##########
 File path: python/tvm/relay/def_use.py
 ##########
 @@ -0,0 +1,42 @@
+from . import expr
+from .expr_functor import ExprVisitor
+from .analysis import free_vars
+
+import attr
+from typing import List, Dict
+
+@attr.s(auto_attribs=True)
+class DefUse:
+    defn: expr.Var
+    uses: List[expr.Expr]
+
+class DefUseAnalysis(ExprVisitor):
+    def __init__(self):
+        super().__init__()
+        self.results: Dict[expr.Var, DefUse] = {}
+
+    def visit_function(self, func):
+        for param in func.params:
+            self.results[param] = DefUse(param, [])
+        super().visit_function(func)
+
+    def visit_let(self, let):
+        while isinstance(let, expr.Let):
+            du = DefUse(let.var, [])
+            self.results[let.var] = du
+            # Find all variables used in RHS.
+            self.visit(let.value)
+            used_vars = free_vars(let.value)
+            for uvar in used_vars:
+                self.results[uvar].uses.append(let.value)
+            let = let.body
+
+        # Find all variables used in body.
+        used_vars = free_vars(let)
 
 Review comment:
   Should this be `free_vars(let.body)`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406977839
 
 

 ##########
 File path: python/tvm/relay/transform/memory_plan.py
 ##########
 @@ -0,0 +1,189 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+import attr
+import numpy as np
+from typing import Optional, Dict
+
+from ..expr_functor import ExprMutator
+from ..scope_builder import ScopeBuilder
+from .. import op, ty, expr
+from ... import DataType, register_func, IRModule
+from .. import analysis
+from . import FoldConstant, InferType, function_pass
+from ..backend import compile_engine
+
+def is_primitive(call):
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
+           hasattr(call.op.attrs, 'Primitive') and int(call.op.attrs.Primitive) == 1
+
+@attr.s(auto_attribs=True)
+class Region:
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    offsets: Dict[expr.Var, expr.Expr] = {}
+
+    def grow(self, old_storage: expr.Var, size: expr.Expr, alignment: expr.Expr, dtype: str) -> None:
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert analysis.alpha_equal(self.alignment, alignment), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        # Record the offset at which we allocate the storage.
+        self.offsets[old_storage] = self.size
+
+        self.size = self.size + size
+
+    def to_expr(self) -> expr.Expr:
+        return op.memory.alloc_storage(self.size, self.alignment, self.dtype)
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        body = expr.Let(var, value, body)
+    return body
+
+class StorageCoalesce(ExprMutator):
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self):
+        zero = expr.const(0, dtype="int64")
+        region_var = expr.var(f"region{len(self.regions)}")
+        region = Region(region_var, zero, None, None)
+        self.regions.append(region)
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        region = self.regions.pop()
+        storage_expr = region.to_expr()
+        assert storage_expr, "can not be None"
+        return expr.Let(region.var, storage_expr, body)
+
+    def current_region(self) -> Region:
+        return self.regions[-1]
+
+    def visit_function(self, function):
+        if function.attrs and int(function.attrs.Primitive) == 1:
+            return super().visit_function(function)
+        else:
+            self.enter_scope()
+            body = self.visit(function.body)
+            body = self.exit_scope(body)
+            return expr.Function(
+                function.params,
+                body,
+                function.ret_type,
+                function.type_params,
+                function.attrs)
+
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+    def visit_let(self, let):
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get("memory.alloc_storage"):
+                return self.process_alloc_storage(lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get("memory.alloc_tensor"):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        return iterative_let(let, _each_binding, mk_let)
+
+    def process_alloc_storage(self, lhs, call):
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        region = self.current_region()
+        region.grow(lhs, size, alignment, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        region = self.current_region()
+        storage, old_offset, shape = call.args
+        offset = region.offsets[storage]
+        assert old_offset.data.asnumpy().item() == 0, "no offsets should yet be allocated"
+        return lhs, expr.Call(call.op, [region.var, offset, shape], call.attrs, call.type_args)
+
+
+class MemoryPlanPass(ExprMutator):
+    """A pass for coalescing allocations made by the Relay VM."""
+    # def visit_let(self, let):
+    #     import pdb; pdb.set_trace()
+    # pass restore after rebase
+
+
+def eval_const(mod, func):
+    mod["tmp"] = func
+    mod = FoldConstant()(mod)
+    return mod["tmp"]
+
+def infer_type(mod, func):
+    mod["tmp"] = func
+    mod = FoldConstant()(mod)
+    return mod["tmp"]
+
+
+@function_pass(opt_level=0)
+class MemoryPlan:
+    """An explicit pass wrapper around ManifestAlloc."""
+    def __init__(self):
+        super().__init__()
+        pass
+
+    def transform_function(self, func, mod, _):
+        # TODO(@jroesch): Is there a way to do one shot initialization, no need to import every time?
+        mod.import_from_std("core.rly")
+        sc = StorageCoalesce()
+        func = sc.visit(func)
+        func = infer_type(mod, func)
+        func = eval_const(mod, func)
+        ea = MemoryPlanPass()
+        func = ea.visit(func)
+        print(func)
 
 Review comment:
   remove the print

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406978657
 
 

 ##########
 File path: python/tvm/relay/def_use.py
 ##########
 @@ -0,0 +1,42 @@
+from . import expr
+from .expr_functor import ExprVisitor
+from .analysis import free_vars
+
+import attr
+from typing import List, Dict
+
+@attr.s(auto_attribs=True)
+class DefUse:
+    defn: expr.Var
+    uses: List[expr.Expr]
+
+class DefUseAnalysis(ExprVisitor):
 
 Review comment:
   not in use right now?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r406978511
 
 

 ##########
 File path: src/runtime/vm/vm.cc
 ##########
 @@ -525,7 +533,8 @@ void InstructionPrint(std::ostream& os, const Instruction& instr) {
     }
     case Opcode::AllocTensor: {
       os << "alloc_tensor $" << instr.dst << " $"
-         << instr.alloc_tensor.storage << " ["
+         << instr.alloc_tensor.storage << " $"
 
 Review comment:
   ```suggestion
            << instr.alloc_tensor.storage << " "
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5144:
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r424747097



##########
File path: python/tvm/relay/transform/memory_plan.py
##########
@@ -0,0 +1,353 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+from typing import Optional, Dict, List, Tuple
+from collections import defaultdict
+import attr
+
+from ..expr_functor import ExprMutator
+from .. import op, expr
+from ..function import Function
+from ... import register_func, ir, cpu
+from ..._ffi.runtime_ctypes import TVMContext
+from ... import IRModule
+from .. import transform
+from . import function_pass
+
+
+def is_primitive(call):
+    return (
+        hasattr(call, "op")
+        and hasattr(call.op, "attrs")
+        and hasattr(call.op.attrs, "Primitive")
+        and int(call.op.attrs.Primitive) == 1
+    )
+
+
+@attr.s(auto_attribs=True)
+class Region:
+    """
+    Represents a control-free allocation region.
+
+    The below pass groups sets of allocations into regions,
+    then replaces the region with a single allocation.
+    """
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    ctx: TVMContext
+    offsets: Dict[expr.Var, Tuple[expr.Expr, expr.Expr]]
+
+    @staticmethod
+    def empty(region_no):
+        zero = expr.const(0, dtype="int64")
+        assert len(zero.data.shape) == 0
+        region_var = expr.var(f"region{region_no}")
+        return Region(region_var, zero, None, None, None, {})
+
+    def grow(
+            self, old_storage: expr.Var,
+            size: expr.Expr, alignment: expr.Expr,
+            ctx: TVMContext,
+            dtype: str) -> None:
+        """Grow the region by a given allocation as well as track the old storage
+           for later rewriting the program to use the allocated region.
+        """
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert ir.structural_equal(
+                self.alignment, alignment
+            ), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        if self.ctx:
+            assert (self.ctx.device_type == ctx.device_type and
+                    self.ctx.device_id == ctx.device_id), "must have matching context"
+        else:
+            assert ctx
+            self.ctx = ctx
+
+        new_size = (size + self.alignment - expr.const(1, "int64")) \
+            / self.alignment * self.alignment
+
+        # Record the offset at which we allocate the storage.
+        offset_var: expr.RelayExpr = expr.var(f"offset{len(self.offsets)}")
+        self.offsets[old_storage] = (offset_var, self.size)
+
+        self.size = self.size + new_size
+
+    def offset_for(self, alloc: expr.Expr) -> expr.Expr:
+        return self.offsets.get(alloc, [None])[0]
+
+    def to_expr(self, body: expr.Expr) -> expr.Expr:
+        """
+        Generate the prelude code for a region, wrapping the body in it.
+
+        The prelude contains the single allocation for a region, and
+        all offset computations.
+        """
+
+        if self.ctx is None:
+            self.ctx = cpu(0)
+
+        # Generate bindings for each and every size computation
+        # we must do this to maintain ANF.
+        bindings: List[Tuple[expr.Expr, expr.Expr]] = []
+
+        # First compute the total size.
+        total_size = expr.var(f"total_size{hash(body)}")
+        bindings.append((total_size, self.size))
+
+        # Allocate the entire region with a single call.
+        alloc = op.memory.alloc_storage(total_size, self.alignment, self.ctx, self.dtype)
+        bindings.append((self.var, alloc))
+
+        # Generate variables which contain all of the offset math.
+        # Ensure we constant evaluate away all the math here.
+        #
+        # In theory we can support dynamic offsets but this
+        # requires another round of memory planning and
+        # potentially colaescing.
+        for alloc in self.offsets:
+            (var, offset) = self.offsets[alloc]
+            bindings.append((var, offset))
+
+        body = mk_let(bindings, body)
+        return body
+
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        assert var
+        assert value
+        assert body
+        body = expr.Let(var, value, body)
+
+    return body
+
+def const_eval(mod, exp):
+    mod = IRModule.from_expr(exp, type_defs=mod.type_definitions)
+    mod = transform.FoldConstant()(mod)
+    return mod["main"]
+
+class StorageCoalesce(ExprMutator):
+    """
+    A pass for coalescing allocations into region/arena allocations.
+
+    After this pass each allocation comes from the same backing storage,
+    but will never overlap even in time, i.e. the allocations are just
+    packed into a contiguous block of memory.
+
+    A secondary part of memory planning will perform liveness analysis to
+    overlap these in time, i.e when an early tensor dies we will attempt
+    to reuse its slot.
+    """
+
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self) -> None:
+        region_no = len(self.regions)
+        self.regions.append(defaultdict(lambda: Region.empty(region_no)))
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        """When leaving a scope build a region allocation for the scope."""
+        dtype_region = self.regions.pop()
+        for _, region in reversed(list(dtype_region.items())):
+            if len(region.offsets) != 0:
+                body = region.to_expr(body)
+
+        return body
+
+    def current_region(self, dtype) -> Region:
+        current_scope = self.regions[-1]
+        return current_scope[dtype]
+
+    def new_region_and_offset(self, old_storage):
+        for dtype_region in reversed(self.regions):
+            for dtype in dtype_region:
+                region = dtype_region[dtype]
+                offset = region.offset_for(old_storage)
+                if offset:
+                    return region, offset
+
+        raise Exception("could not find offset in any valid region")
+
+    def visit_function(self, fn):
+        """Transform the function body to use region allocation scheme."""
+        func = fn
+        if getattr(func.attrs, "Primitive", 0) == 1:
+            return super().visit_function(func)
+        else:
+            self.enter_scope()
+            body = self.visit(func.body)
+            body = self.exit_scope(body)
+            return Function(
+                func.params,
+                body,
+                func.ret_type,
+                func.type_params,
+                func.attrs,
+            )
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+
+    def mk_let(self, dynamic_regions):
+        """Let bind the dynamic regions"""
+        def _mk_let(bindings, body):
+            for var, value in reversed(bindings):
+                assert var
+                assert value
+                assert body
+                body = expr.Let(var, value, body)
+                if var in dynamic_regions:
+                    body = self.exit_scope(body)
+
+            return body
+
+        return _mk_let
+
+    def visit_let(self, let):
+        dynamic_regions = []
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_storage"
+            ):
+                return self.process_alloc_storage(dynamic_regions, lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_tensor"
+            ):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        result = iterative_let(let, _each_binding, self.mk_let(dynamic_regions))
+        assert result
+        return result
+
+    def process_alloc_storage(self, dynamic_regions, lhs, call):
+        """Process alloc_storage"""
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        ctx = TVMContext(call.attrs.device_type, call.attrs.device_id)
+
+        if not isinstance(size, expr.Constant):
+            self.enter_scope()
+            dynamic_regions.append(lhs)
+
+        region = self.current_region(dtype)
+        region.grow(lhs, size, alignment, ctx, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        """Process alloc tensor. Region and offset are computed"""
+        storage, old_offset, shape = call.args
+        region, offset = self.new_region_and_offset(storage)
+
+        assert (
+            old_offset.data.asnumpy().item() == 0
+        ), "no offsets should yet be allocated"
+        return (
+            lhs,
+            expr.Call(call.op, [region.var, offset, shape], call.attrs),
+        )
+
+class LiftConst(ExprMutator):
+    """A internal pass to lift constants to the top level of function."""
+    def __init__(self):
+        self.i = 0
+        self.constants = []
+        self.top_level = True
+        super().__init__()
+
+    def visit_constant(self, const):
+        var = expr.var(f"const{self.i}")
+        self.i += 1
+        self.constants.append((var, const))
+        return var
+
+    def visit_function(self, fn):
+        if int(getattr(fn.attrs, "Primitive", 0)) == 1:
+            return fn
+
+        outer_constant = self.constants
+        self.constants = []
+        body = mk_let(self.constants, self.visit(fn.body))
+        self.constants = outer_constant
+
+        return Function(
+            fn.params,
+            body,
+            fn.ret_type,
+            fn.type_params,
+            fn.attrs)
+
+@function_pass(opt_level=0)
+class MemoryPlan:
+    """An explicit pass wrapper around ManifestAlloc."""

Review comment:
       ```suggestion
       """An explicit pass wrapper around StorageCoalesce."""
   ```

##########
File path: src/runtime/vm/vm.cc
##########
@@ -610,6 +619,37 @@ inline ObjectRef CopyTo(ObjectRef src, const DLContext& ctx) {
   return src;
 }
 
+std::vector<int64_t> ToShape(NDArray shape_tensor) {
+  std::vector<int64_t> shape;
+  auto rank = shape_tensor.Shape().size();
+  auto dtype = shape_tensor.DataType();
+
+  // For 0-rank shapes we need to allocate a single scalar.
+  if (rank == 0) {
+    return shape;
+  }
+
+  // Otherwise we should be rank-1, and we will extract the number of dimensions
+  // for the output vector.
+  CHECK(shape_tensor.Shape().size() == 1)

Review comment:
       CHECK_EQ(rank, 1U)

##########
File path: python/tvm/relay/transform/memory_plan.py
##########
@@ -0,0 +1,353 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+from typing import Optional, Dict, List, Tuple
+from collections import defaultdict
+import attr
+
+from ..expr_functor import ExprMutator
+from .. import op, expr
+from ..function import Function
+from ... import register_func, ir, cpu
+from ..._ffi.runtime_ctypes import TVMContext
+from ... import IRModule
+from .. import transform
+from . import function_pass
+
+
+def is_primitive(call):
+    return (
+        hasattr(call, "op")
+        and hasattr(call.op, "attrs")
+        and hasattr(call.op.attrs, "Primitive")
+        and int(call.op.attrs.Primitive) == 1
+    )
+
+
+@attr.s(auto_attribs=True)
+class Region:
+    """
+    Represents a control-free allocation region.
+
+    The below pass groups sets of allocations into regions,
+    then replaces the region with a single allocation.
+    """
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    ctx: TVMContext
+    offsets: Dict[expr.Var, Tuple[expr.Expr, expr.Expr]]
+
+    @staticmethod
+    def empty(region_no):
+        zero = expr.const(0, dtype="int64")
+        assert len(zero.data.shape) == 0
+        region_var = expr.var(f"region{region_no}")
+        return Region(region_var, zero, None, None, None, {})
+
+    def grow(
+            self, old_storage: expr.Var,
+            size: expr.Expr, alignment: expr.Expr,
+            ctx: TVMContext,
+            dtype: str) -> None:
+        """Grow the region by a given allocation as well as track the old storage
+           for later rewriting the program to use the allocated region.
+        """
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert ir.structural_equal(
+                self.alignment, alignment
+            ), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        if self.ctx:
+            assert (self.ctx.device_type == ctx.device_type and
+                    self.ctx.device_id == ctx.device_id), "must have matching context"
+        else:
+            assert ctx
+            self.ctx = ctx
+
+        new_size = (size + self.alignment - expr.const(1, "int64")) \
+            / self.alignment * self.alignment
+
+        # Record the offset at which we allocate the storage.
+        offset_var: expr.RelayExpr = expr.var(f"offset{len(self.offsets)}")
+        self.offsets[old_storage] = (offset_var, self.size)
+
+        self.size = self.size + new_size
+
+    def offset_for(self, alloc: expr.Expr) -> expr.Expr:
+        return self.offsets.get(alloc, [None])[0]
+
+    def to_expr(self, body: expr.Expr) -> expr.Expr:
+        """
+        Generate the prelude code for a region, wrapping the body in it.
+
+        The prelude contains the single allocation for a region, and
+        all offset computations.
+        """
+
+        if self.ctx is None:
+            self.ctx = cpu(0)
+
+        # Generate bindings for each and every size computation
+        # we must do this to maintain ANF.
+        bindings: List[Tuple[expr.Expr, expr.Expr]] = []
+
+        # First compute the total size.
+        total_size = expr.var(f"total_size{hash(body)}")
+        bindings.append((total_size, self.size))
+
+        # Allocate the entire region with a single call.
+        alloc = op.memory.alloc_storage(total_size, self.alignment, self.ctx, self.dtype)
+        bindings.append((self.var, alloc))
+
+        # Generate variables which contain all of the offset math.
+        # Ensure we constant evaluate away all the math here.
+        #
+        # In theory we can support dynamic offsets but this
+        # requires another round of memory planning and
+        # potentially colaescing.
+        for alloc in self.offsets:
+            (var, offset) = self.offsets[alloc]
+            bindings.append((var, offset))
+
+        body = mk_let(bindings, body)
+        return body
+
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        assert var
+        assert value
+        assert body
+        body = expr.Let(var, value, body)
+
+    return body
+
+def const_eval(mod, exp):
+    mod = IRModule.from_expr(exp, type_defs=mod.type_definitions)
+    mod = transform.FoldConstant()(mod)
+    return mod["main"]
+
+class StorageCoalesce(ExprMutator):
+    """
+    A pass for coalescing allocations into region/arena allocations.
+
+    After this pass each allocation comes from the same backing storage,
+    but will never overlap even in time, i.e. the allocations are just
+    packed into a contiguous block of memory.
+
+    A secondary part of memory planning will perform liveness analysis to
+    overlap these in time, i.e when an early tensor dies we will attempt
+    to reuse its slot.
+    """
+
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self) -> None:
+        region_no = len(self.regions)
+        self.regions.append(defaultdict(lambda: Region.empty(region_no)))
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        """When leaving a scope build a region allocation for the scope."""
+        dtype_region = self.regions.pop()
+        for _, region in reversed(list(dtype_region.items())):
+            if len(region.offsets) != 0:
+                body = region.to_expr(body)
+
+        return body
+
+    def current_region(self, dtype) -> Region:
+        current_scope = self.regions[-1]
+        return current_scope[dtype]
+
+    def new_region_and_offset(self, old_storage):
+        for dtype_region in reversed(self.regions):
+            for dtype in dtype_region:
+                region = dtype_region[dtype]
+                offset = region.offset_for(old_storage)
+                if offset:
+                    return region, offset
+
+        raise Exception("could not find offset in any valid region")
+
+    def visit_function(self, fn):
+        """Transform the function body to use region allocation scheme."""
+        func = fn
+        if getattr(func.attrs, "Primitive", 0) == 1:
+            return super().visit_function(func)
+        else:
+            self.enter_scope()
+            body = self.visit(func.body)
+            body = self.exit_scope(body)
+            return Function(
+                func.params,
+                body,
+                func.ret_type,
+                func.type_params,
+                func.attrs,
+            )
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+
+    def mk_let(self, dynamic_regions):
+        """Let bind the dynamic regions"""
+        def _mk_let(bindings, body):
+            for var, value in reversed(bindings):
+                assert var
+                assert value
+                assert body
+                body = expr.Let(var, value, body)
+                if var in dynamic_regions:
+                    body = self.exit_scope(body)
+
+            return body
+
+        return _mk_let
+
+    def visit_let(self, let):
+        dynamic_regions = []
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_storage"
+            ):
+                return self.process_alloc_storage(dynamic_regions, lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_tensor"
+            ):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        result = iterative_let(let, _each_binding, self.mk_let(dynamic_regions))
+        assert result
+        return result
+
+    def process_alloc_storage(self, dynamic_regions, lhs, call):
+        """Process alloc_storage"""
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        ctx = TVMContext(call.attrs.device_type, call.attrs.device_id)
+
+        if not isinstance(size, expr.Constant):
+            self.enter_scope()
+            dynamic_regions.append(lhs)
+
+        region = self.current_region(dtype)
+        region.grow(lhs, size, alignment, ctx, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        """Process alloc tensor. Region and offset are computed"""
+        storage, old_offset, shape = call.args
+        region, offset = self.new_region_and_offset(storage)
+
+        assert (
+            old_offset.data.asnumpy().item() == 0
+        ), "no offsets should yet be allocated"
+        return (
+            lhs,
+            expr.Call(call.op, [region.var, offset, shape], call.attrs),
+        )
+
+class LiftConst(ExprMutator):
+    """A internal pass to lift constants to the top level of function."""
+    def __init__(self):
+        self.i = 0
+        self.constants = []
+        self.top_level = True
+        super().__init__()
+
+    def visit_constant(self, const):
+        var = expr.var(f"const{self.i}")
+        self.i += 1
+        self.constants.append((var, const))
+        return var
+
+    def visit_function(self, fn):
+        if int(getattr(fn.attrs, "Primitive", 0)) == 1:
+            return fn
+
+        outer_constant = self.constants
+        self.constants = []
+        body = mk_let(self.constants, self.visit(fn.body))

Review comment:
       would this `mk_let` do anything as `self.constants` is always empty? Or should `outer_constant` be passed?

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -56,10 +57,22 @@ Pass InlinePrimitives();
 
 Pass ManifestAlloc(Target target_host) {
   auto f = tvm::runtime::Registry::Get("relay.transform.ManifestAlloc");
-  CHECK(f != nullptr) << "could not load memory allocation pass";
+  CHECK(f != nullptr) << "unable to load allocation manifestation pass";
   return (*f)(target_host);
 }
 
+Pass MemoryPlan() {
+  auto f = tvm::runtime::Registry::Get("relay.transform.MemoryPlan");
+  CHECK(f != nullptr) << "unable to load the memory planning pass";
+  return (*f)();
+}
+
+Pass LiftConstants() {
+  auto f = tvm::runtime::Registry::Get("relay.transform.LiftConstants");
+  CHECK(f != nullptr) << "unable to load the memory planning pass";

Review comment:
       ```suggestion
     CHECK(f != nullptr) << "unable to load the constant lifting pass";
   ```

##########
File path: python/tvm/relay/transform/memory_plan.py
##########
@@ -0,0 +1,353 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+from typing import Optional, Dict, List, Tuple
+from collections import defaultdict
+import attr
+
+from ..expr_functor import ExprMutator
+from .. import op, expr
+from ..function import Function
+from ... import register_func, ir, cpu
+from ..._ffi.runtime_ctypes import TVMContext
+from ... import IRModule
+from .. import transform
+from . import function_pass
+
+
+def is_primitive(call):
+    return (
+        hasattr(call, "op")
+        and hasattr(call.op, "attrs")
+        and hasattr(call.op.attrs, "Primitive")
+        and int(call.op.attrs.Primitive) == 1
+    )
+
+
+@attr.s(auto_attribs=True)
+class Region:
+    """
+    Represents a control-free allocation region.
+
+    The below pass groups sets of allocations into regions,
+    then replaces the region with a single allocation.
+    """
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    ctx: TVMContext
+    offsets: Dict[expr.Var, Tuple[expr.Expr, expr.Expr]]
+
+    @staticmethod
+    def empty(region_no):
+        zero = expr.const(0, dtype="int64")
+        assert len(zero.data.shape) == 0
+        region_var = expr.var(f"region{region_no}")
+        return Region(region_var, zero, None, None, None, {})
+
+    def grow(
+            self, old_storage: expr.Var,
+            size: expr.Expr, alignment: expr.Expr,
+            ctx: TVMContext,
+            dtype: str) -> None:
+        """Grow the region by a given allocation as well as track the old storage
+           for later rewriting the program to use the allocated region.
+        """
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert ir.structural_equal(
+                self.alignment, alignment
+            ), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        if self.ctx:
+            assert (self.ctx.device_type == ctx.device_type and
+                    self.ctx.device_id == ctx.device_id), "must have matching context"
+        else:
+            assert ctx
+            self.ctx = ctx
+
+        new_size = (size + self.alignment - expr.const(1, "int64")) \
+            / self.alignment * self.alignment
+
+        # Record the offset at which we allocate the storage.
+        offset_var: expr.RelayExpr = expr.var(f"offset{len(self.offsets)}")
+        self.offsets[old_storage] = (offset_var, self.size)
+
+        self.size = self.size + new_size
+
+    def offset_for(self, alloc: expr.Expr) -> expr.Expr:
+        return self.offsets.get(alloc, [None])[0]
+
+    def to_expr(self, body: expr.Expr) -> expr.Expr:
+        """
+        Generate the prelude code for a region, wrapping the body in it.
+
+        The prelude contains the single allocation for a region, and
+        all offset computations.
+        """
+
+        if self.ctx is None:
+            self.ctx = cpu(0)
+
+        # Generate bindings for each and every size computation
+        # we must do this to maintain ANF.
+        bindings: List[Tuple[expr.Expr, expr.Expr]] = []
+
+        # First compute the total size.
+        total_size = expr.var(f"total_size{hash(body)}")
+        bindings.append((total_size, self.size))
+
+        # Allocate the entire region with a single call.
+        alloc = op.memory.alloc_storage(total_size, self.alignment, self.ctx, self.dtype)
+        bindings.append((self.var, alloc))
+
+        # Generate variables which contain all of the offset math.
+        # Ensure we constant evaluate away all the math here.
+        #
+        # In theory we can support dynamic offsets but this
+        # requires another round of memory planning and
+        # potentially colaescing.
+        for alloc in self.offsets:
+            (var, offset) = self.offsets[alloc]
+            bindings.append((var, offset))
+
+        body = mk_let(bindings, body)
+        return body
+
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        assert var
+        assert value
+        assert body
+        body = expr.Let(var, value, body)
+
+    return body
+
+def const_eval(mod, exp):
+    mod = IRModule.from_expr(exp, type_defs=mod.type_definitions)
+    mod = transform.FoldConstant()(mod)
+    return mod["main"]
+
+class StorageCoalesce(ExprMutator):
+    """
+    A pass for coalescing allocations into region/arena allocations.
+
+    After this pass each allocation comes from the same backing storage,
+    but will never overlap even in time, i.e. the allocations are just
+    packed into a contiguous block of memory.
+
+    A secondary part of memory planning will perform liveness analysis to
+    overlap these in time, i.e when an early tensor dies we will attempt
+    to reuse its slot.
+    """
+
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self) -> None:
+        region_no = len(self.regions)
+        self.regions.append(defaultdict(lambda: Region.empty(region_no)))
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        """When leaving a scope build a region allocation for the scope."""
+        dtype_region = self.regions.pop()
+        for _, region in reversed(list(dtype_region.items())):
+            if len(region.offsets) != 0:
+                body = region.to_expr(body)
+
+        return body
+
+    def current_region(self, dtype) -> Region:
+        current_scope = self.regions[-1]
+        return current_scope[dtype]
+
+    def new_region_and_offset(self, old_storage):
+        for dtype_region in reversed(self.regions):
+            for dtype in dtype_region:
+                region = dtype_region[dtype]
+                offset = region.offset_for(old_storage)
+                if offset:
+                    return region, offset
+
+        raise Exception("could not find offset in any valid region")
+
+    def visit_function(self, fn):
+        """Transform the function body to use region allocation scheme."""
+        func = fn
+        if getattr(func.attrs, "Primitive", 0) == 1:
+            return super().visit_function(func)
+        else:
+            self.enter_scope()
+            body = self.visit(func.body)
+            body = self.exit_scope(body)
+            return Function(
+                func.params,
+                body,
+                func.ret_type,
+                func.type_params,
+                func.attrs,
+            )
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+
+    def mk_let(self, dynamic_regions):
+        """Let bind the dynamic regions"""
+        def _mk_let(bindings, body):
+            for var, value in reversed(bindings):
+                assert var
+                assert value
+                assert body
+                body = expr.Let(var, value, body)
+                if var in dynamic_regions:
+                    body = self.exit_scope(body)
+
+            return body
+
+        return _mk_let
+
+    def visit_let(self, let):
+        dynamic_regions = []
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_storage"
+            ):
+                return self.process_alloc_storage(dynamic_regions, lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_tensor"
+            ):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        result = iterative_let(let, _each_binding, self.mk_let(dynamic_regions))
+        assert result
+        return result
+
+    def process_alloc_storage(self, dynamic_regions, lhs, call):
+        """Process alloc_storage"""
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        ctx = TVMContext(call.attrs.device_type, call.attrs.device_id)
+
+        if not isinstance(size, expr.Constant):
+            self.enter_scope()
+            dynamic_regions.append(lhs)
+
+        region = self.current_region(dtype)
+        region.grow(lhs, size, alignment, ctx, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        """Process alloc tensor. Region and offset are computed"""
+        storage, old_offset, shape = call.args
+        region, offset = self.new_region_and_offset(storage)
+
+        assert (
+            old_offset.data.asnumpy().item() == 0
+        ), "no offsets should yet be allocated"
+        return (
+            lhs,
+            expr.Call(call.op, [region.var, offset, shape], call.attrs),
+        )
+
+class LiftConst(ExprMutator):
+    """A internal pass to lift constants to the top level of function."""
+    def __init__(self):
+        self.i = 0
+        self.constants = []
+        self.top_level = True
+        super().__init__()
+
+    def visit_constant(self, const):
+        var = expr.var(f"const{self.i}")
+        self.i += 1
+        self.constants.append((var, const))
+        return var
+
+    def visit_function(self, fn):
+        if int(getattr(fn.attrs, "Primitive", 0)) == 1:
+            return fn
+
+        outer_constant = self.constants
+        self.constants = []
+        body = mk_let(self.constants, self.visit(fn.body))
+        self.constants = outer_constant
+
+        return Function(
+            fn.params,
+            body,
+            fn.ret_type,
+            fn.type_params,
+            fn.attrs)
+
+@function_pass(opt_level=0)
+class MemoryPlan:
+    """An explicit pass wrapper around ManifestAlloc."""
+
+    def transform_function(self, func, mod, _):
+        mod.import_from_std("core.rly")
+        sc = StorageCoalesce()
+        func = sc.visit(func)
+        return func
+
+register_func("relay.transform.MemoryPlan", MemoryPlan)
+
+@function_pass(opt_level=0)
+class LiftConstants:
+    """An explicit pass wrapper around LiftConstants."""

Review comment:
       ```suggestion
       """An explicit pass wrapper around LiftConst."""
   ```

##########
File path: python/tvm/relay/transform/memory_plan.py
##########
@@ -0,0 +1,353 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+from typing import Optional, Dict, List, Tuple
+from collections import defaultdict
+import attr
+
+from ..expr_functor import ExprMutator
+from .. import op, expr
+from ..function import Function
+from ... import register_func, ir, cpu
+from ..._ffi.runtime_ctypes import TVMContext
+from ... import IRModule
+from .. import transform
+from . import function_pass
+
+
+def is_primitive(call):
+    return (
+        hasattr(call, "op")
+        and hasattr(call.op, "attrs")
+        and hasattr(call.op.attrs, "Primitive")
+        and int(call.op.attrs.Primitive) == 1
+    )
+
+
+@attr.s(auto_attribs=True)
+class Region:
+    """
+    Represents a control-free allocation region.
+
+    The below pass groups sets of allocations into regions,
+    then replaces the region with a single allocation.
+    """
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    ctx: TVMContext
+    offsets: Dict[expr.Var, Tuple[expr.Expr, expr.Expr]]
+
+    @staticmethod
+    def empty(region_no):
+        zero = expr.const(0, dtype="int64")
+        assert len(zero.data.shape) == 0
+        region_var = expr.var(f"region{region_no}")
+        return Region(region_var, zero, None, None, None, {})
+
+    def grow(
+            self, old_storage: expr.Var,
+            size: expr.Expr, alignment: expr.Expr,
+            ctx: TVMContext,
+            dtype: str) -> None:
+        """Grow the region by a given allocation as well as track the old storage
+           for later rewriting the program to use the allocated region.
+        """
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert ir.structural_equal(
+                self.alignment, alignment
+            ), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        if self.ctx:
+            assert (self.ctx.device_type == ctx.device_type and
+                    self.ctx.device_id == ctx.device_id), "must have matching context"
+        else:
+            assert ctx
+            self.ctx = ctx
+
+        new_size = (size + self.alignment - expr.const(1, "int64")) \
+            / self.alignment * self.alignment
+
+        # Record the offset at which we allocate the storage.
+        offset_var: expr.RelayExpr = expr.var(f"offset{len(self.offsets)}")
+        self.offsets[old_storage] = (offset_var, self.size)
+
+        self.size = self.size + new_size
+
+    def offset_for(self, alloc: expr.Expr) -> expr.Expr:
+        return self.offsets.get(alloc, [None])[0]
+
+    def to_expr(self, body: expr.Expr) -> expr.Expr:
+        """
+        Generate the prelude code for a region, wrapping the body in it.
+
+        The prelude contains the single allocation for a region, and
+        all offset computations.
+        """
+
+        if self.ctx is None:
+            self.ctx = cpu(0)
+
+        # Generate bindings for each and every size computation
+        # we must do this to maintain ANF.
+        bindings: List[Tuple[expr.Expr, expr.Expr]] = []
+
+        # First compute the total size.
+        total_size = expr.var(f"total_size{hash(body)}")
+        bindings.append((total_size, self.size))
+
+        # Allocate the entire region with a single call.
+        alloc = op.memory.alloc_storage(total_size, self.alignment, self.ctx, self.dtype)
+        bindings.append((self.var, alloc))
+
+        # Generate variables which contain all of the offset math.
+        # Ensure we constant evaluate away all the math here.
+        #
+        # In theory we can support dynamic offsets but this
+        # requires another round of memory planning and
+        # potentially colaescing.
+        for alloc in self.offsets:
+            (var, offset) = self.offsets[alloc]
+            bindings.append((var, offset))
+
+        body = mk_let(bindings, body)
+        return body
+
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        assert var
+        assert value
+        assert body
+        body = expr.Let(var, value, body)
+
+    return body
+
+def const_eval(mod, exp):
+    mod = IRModule.from_expr(exp, type_defs=mod.type_definitions)
+    mod = transform.FoldConstant()(mod)
+    return mod["main"]
+
+class StorageCoalesce(ExprMutator):
+    """
+    A pass for coalescing allocations into region/arena allocations.
+
+    After this pass each allocation comes from the same backing storage,
+    but will never overlap even in time, i.e. the allocations are just
+    packed into a contiguous block of memory.
+
+    A secondary part of memory planning will perform liveness analysis to
+    overlap these in time, i.e when an early tensor dies we will attempt
+    to reuse its slot.
+    """
+
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self) -> None:
+        region_no = len(self.regions)
+        self.regions.append(defaultdict(lambda: Region.empty(region_no)))
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        """When leaving a scope build a region allocation for the scope."""
+        dtype_region = self.regions.pop()
+        for _, region in reversed(list(dtype_region.items())):
+            if len(region.offsets) != 0:
+                body = region.to_expr(body)
+
+        return body
+
+    def current_region(self, dtype) -> Region:
+        current_scope = self.regions[-1]
+        return current_scope[dtype]
+
+    def new_region_and_offset(self, old_storage):
+        for dtype_region in reversed(self.regions):
+            for dtype in dtype_region:
+                region = dtype_region[dtype]
+                offset = region.offset_for(old_storage)
+                if offset:
+                    return region, offset
+
+        raise Exception("could not find offset in any valid region")
+
+    def visit_function(self, fn):
+        """Transform the function body to use region allocation scheme."""
+        func = fn
+        if getattr(func.attrs, "Primitive", 0) == 1:
+            return super().visit_function(func)
+        else:
+            self.enter_scope()
+            body = self.visit(func.body)
+            body = self.exit_scope(body)
+            return Function(
+                func.params,
+                body,
+                func.ret_type,
+                func.type_params,
+                func.attrs,
+            )
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+
+    def mk_let(self, dynamic_regions):
+        """Let bind the dynamic regions"""
+        def _mk_let(bindings, body):
+            for var, value in reversed(bindings):
+                assert var
+                assert value
+                assert body
+                body = expr.Let(var, value, body)
+                if var in dynamic_regions:
+                    body = self.exit_scope(body)
+
+            return body
+
+        return _mk_let
+
+    def visit_let(self, let):
+        dynamic_regions = []
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_storage"
+            ):
+                return self.process_alloc_storage(dynamic_regions, lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_tensor"
+            ):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        result = iterative_let(let, _each_binding, self.mk_let(dynamic_regions))
+        assert result
+        return result
+
+    def process_alloc_storage(self, dynamic_regions, lhs, call):
+        """Process alloc_storage"""
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        ctx = TVMContext(call.attrs.device_type, call.attrs.device_id)
+
+        if not isinstance(size, expr.Constant):
+            self.enter_scope()
+            dynamic_regions.append(lhs)
+
+        region = self.current_region(dtype)
+        region.grow(lhs, size, alignment, ctx, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        """Process alloc tensor. Region and offset are computed"""
+        storage, old_offset, shape = call.args
+        region, offset = self.new_region_and_offset(storage)
+
+        assert (
+            old_offset.data.asnumpy().item() == 0
+        ), "no offsets should yet be allocated"
+        return (
+            lhs,
+            expr.Call(call.op, [region.var, offset, shape], call.attrs),
+        )
+
+class LiftConst(ExprMutator):
+    """A internal pass to lift constants to the top level of function."""

Review comment:
       ```suggestion
       """An internal pass to lift constants to the top level of function."""
   ```

##########
File path: src/runtime/vm/vm.cc
##########
@@ -610,6 +619,37 @@ inline ObjectRef CopyTo(ObjectRef src, const DLContext& ctx) {
   return src;
 }
 
+std::vector<int64_t> ToShape(NDArray shape_tensor) {
+  std::vector<int64_t> shape;
+  auto rank = shape_tensor.Shape().size();
+  auto dtype = shape_tensor.DataType();
+
+  // For 0-rank shapes we need to allocate a single scalar.
+  if (rank == 0) {
+    return shape;
+  }
+
+  // Otherwise we should be rank-1, and we will extract the number of dimensions
+  // for the output vector.
+  CHECK(shape_tensor.Shape().size() == 1)
+      << "shape tensor should be a k-length vector, found " << shape_tensor.Shape().size();

Review comment:
       ```suggestion
         << "shape tensor should be a k-length vector, found " << rank;
   ```

##########
File path: src/runtime/vm/executable.cc
##########
@@ -549,39 +550,41 @@ Instruction DeserializeInstruction(const VMInstructionSerializer& instr) {
       return Instruction::InvokePacked(packed_index, arity, output_size, args);
     }
     case Opcode::AllocTensor: {
-      // Number of fields = 6 + instr.alloc_tensor.ndim
-      DCHECK_GE(instr.fields.size(), 6U);
-      DCHECK_EQ(instr.fields.size(), 6U + static_cast<size_t>(instr.fields[4]));
+      // Number of fields = 7 + instr.alloc_tensor.ndim
+      DCHECK_GE(instr.fields.size(), 7U);
+      DCHECK_EQ(instr.fields.size(), 7U + static_cast<size_t>(instr.fields[4]));
 
       RegName storage_reg = instr.fields[0];
+      RegName offset = instr.fields[1];
 
       DLDataType dtype;
-      dtype.code = instr.fields[1];
-      dtype.bits = instr.fields[2];
-      dtype.lanes = instr.fields[3];
+      dtype.code = instr.fields[2];
+      dtype.bits = instr.fields[3];
+      dtype.lanes = instr.fields[4];
 
-      Index ndim = instr.fields[4];
-      RegName dst = instr.fields[5];
+      Index ndim = instr.fields[5];
+      RegName dst = instr.fields[6];
 
-      std::vector<Index> shape = ExtractFields(instr.fields, 6, ndim);
+      std::vector<Index> shape = ExtractFields(instr.fields, 7, ndim);
 
-      return Instruction::AllocTensor(storage_reg, shape, dtype, dst);
+      return Instruction::AllocTensor(storage_reg, offset, shape, dtype, dst);
     }
     case Opcode::AllocTensorReg: {
       // Number of fields = 5

Review comment:
       ```suggestion
         // Number of fields = 7
   ```




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

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



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5144:
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r424823515



##########
File path: python/tvm/relay/transform/memory_plan.py
##########
@@ -0,0 +1,353 @@
+# 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.
+# pylint: disable=no-else-return,invalid-name,len-as-condition,too-many-nested-blocks
+"""
+A pass for manifesting explicit memory allocations.
+"""
+from typing import Optional, Dict, List, Tuple
+from collections import defaultdict
+import attr
+
+from ..expr_functor import ExprMutator
+from .. import op, expr
+from ..function import Function
+from ... import register_func, ir, cpu
+from ..._ffi.runtime_ctypes import TVMContext
+from ... import IRModule
+from .. import transform
+from . import function_pass
+
+
+def is_primitive(call):
+    return (
+        hasattr(call, "op")
+        and hasattr(call.op, "attrs")
+        and hasattr(call.op.attrs, "Primitive")
+        and int(call.op.attrs.Primitive) == 1
+    )
+
+
+@attr.s(auto_attribs=True)
+class Region:
+    """
+    Represents a control-free allocation region.
+
+    The below pass groups sets of allocations into regions,
+    then replaces the region with a single allocation.
+    """
+    var: expr.Var
+    size: expr.Expr
+    alignment: Optional[expr.Expr]
+    dtype: Optional[str]
+    ctx: TVMContext
+    offsets: Dict[expr.Var, Tuple[expr.Expr, expr.Expr]]
+
+    @staticmethod
+    def empty(region_no):
+        zero = expr.const(0, dtype="int64")
+        assert len(zero.data.shape) == 0
+        region_var = expr.var(f"region{region_no}")
+        return Region(region_var, zero, None, None, None, {})
+
+    def grow(
+            self, old_storage: expr.Var,
+            size: expr.Expr, alignment: expr.Expr,
+            ctx: TVMContext,
+            dtype: str) -> None:
+        """Grow the region by a given allocation as well as track the old storage
+           for later rewriting the program to use the allocated region.
+        """
+        if self.dtype:
+            assert self.dtype == dtype, "must have matching dtypes in a region"
+        else:
+            self.dtype = dtype
+
+        if self.alignment:
+            assert ir.structural_equal(
+                self.alignment, alignment
+            ), "must have matching alignments in a region"
+        else:
+            self.alignment = alignment
+
+        if self.ctx:
+            assert (self.ctx.device_type == ctx.device_type and
+                    self.ctx.device_id == ctx.device_id), "must have matching context"
+        else:
+            assert ctx
+            self.ctx = ctx
+
+        new_size = (size + self.alignment - expr.const(1, "int64")) \
+            / self.alignment * self.alignment
+
+        # Record the offset at which we allocate the storage.
+        offset_var: expr.RelayExpr = expr.var(f"offset{len(self.offsets)}")
+        self.offsets[old_storage] = (offset_var, self.size)
+
+        self.size = self.size + new_size
+
+    def offset_for(self, alloc: expr.Expr) -> expr.Expr:
+        return self.offsets.get(alloc, [None])[0]
+
+    def to_expr(self, body: expr.Expr) -> expr.Expr:
+        """
+        Generate the prelude code for a region, wrapping the body in it.
+
+        The prelude contains the single allocation for a region, and
+        all offset computations.
+        """
+
+        if self.ctx is None:
+            self.ctx = cpu(0)
+
+        # Generate bindings for each and every size computation
+        # we must do this to maintain ANF.
+        bindings: List[Tuple[expr.Expr, expr.Expr]] = []
+
+        # First compute the total size.
+        total_size = expr.var(f"total_size{hash(body)}")
+        bindings.append((total_size, self.size))
+
+        # Allocate the entire region with a single call.
+        alloc = op.memory.alloc_storage(total_size, self.alignment, self.ctx, self.dtype)
+        bindings.append((self.var, alloc))
+
+        # Generate variables which contain all of the offset math.
+        # Ensure we constant evaluate away all the math here.
+        #
+        # In theory we can support dynamic offsets but this
+        # requires another round of memory planning and
+        # potentially colaescing.
+        for alloc in self.offsets:
+            (var, offset) = self.offsets[alloc]
+            bindings.append((var, offset))
+
+        body = mk_let(bindings, body)
+        return body
+
+
+def iterative_let(let, each_binding, kont):
+    bindings = []
+    while isinstance(let, expr.Let):
+        lhs = let.var
+        rhs = let.value
+        bindings.append(each_binding(lhs, rhs))
+        let = let.body
+
+    return kont(bindings, let)
+
+
+
+def mk_let(bindings, body):
+    for var, value in reversed(bindings):
+        assert var
+        assert value
+        assert body
+        body = expr.Let(var, value, body)
+
+    return body
+
+def const_eval(mod, exp):
+    mod = IRModule.from_expr(exp, type_defs=mod.type_definitions)
+    mod = transform.FoldConstant()(mod)
+    return mod["main"]
+
+class StorageCoalesce(ExprMutator):
+    """
+    A pass for coalescing allocations into region/arena allocations.
+
+    After this pass each allocation comes from the same backing storage,
+    but will never overlap even in time, i.e. the allocations are just
+    packed into a contiguous block of memory.
+
+    A secondary part of memory planning will perform liveness analysis to
+    overlap these in time, i.e when an early tensor dies we will attempt
+    to reuse its slot.
+    """
+
+    def __init__(self):
+        super().__init__()
+        self.regions = []
+
+    def enter_scope(self) -> None:
+        region_no = len(self.regions)
+        self.regions.append(defaultdict(lambda: Region.empty(region_no)))
+
+    def exit_scope(self, body: expr.Expr) -> expr.Expr:
+        """When leaving a scope build a region allocation for the scope."""
+        dtype_region = self.regions.pop()
+        for _, region in reversed(list(dtype_region.items())):
+            if len(region.offsets) != 0:
+                body = region.to_expr(body)
+
+        return body
+
+    def current_region(self, dtype) -> Region:
+        current_scope = self.regions[-1]
+        return current_scope[dtype]
+
+    def new_region_and_offset(self, old_storage):
+        for dtype_region in reversed(self.regions):
+            for dtype in dtype_region:
+                region = dtype_region[dtype]
+                offset = region.offset_for(old_storage)
+                if offset:
+                    return region, offset
+
+        raise Exception("could not find offset in any valid region")
+
+    def visit_function(self, fn):
+        """Transform the function body to use region allocation scheme."""
+        func = fn
+        if getattr(func.attrs, "Primitive", 0) == 1:
+            return super().visit_function(func)
+        else:
+            self.enter_scope()
+            body = self.visit(func.body)
+            body = self.exit_scope(body)
+            return Function(
+                func.params,
+                body,
+                func.ret_type,
+                func.type_params,
+                func.attrs,
+            )
+
+    def visit_if(self, ite):
+        self.enter_scope()
+        true_branch = self.visit(ite.true_branch)
+        true_branch = self.exit_scope(true_branch)
+
+        self.enter_scope()
+        false_branch = self.visit(ite.false_branch)
+        false_branch = self.exit_scope(false_branch)
+
+        return expr.If(ite.cond, true_branch, false_branch)
+
+
+    def mk_let(self, dynamic_regions):
+        """Let bind the dynamic regions"""
+        def _mk_let(bindings, body):
+            for var, value in reversed(bindings):
+                assert var
+                assert value
+                assert body
+                body = expr.Let(var, value, body)
+                if var in dynamic_regions:
+                    body = self.exit_scope(body)
+
+            return body
+
+        return _mk_let
+
+    def visit_let(self, let):
+        dynamic_regions = []
+        def _each_binding(lhs, rhs):
+            if isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_storage"
+            ):
+                return self.process_alloc_storage(dynamic_regions, lhs, rhs)
+            elif isinstance(rhs, expr.Call) and rhs.op == op.op.get(
+                    "memory.alloc_tensor"
+            ):
+                return self.process_alloc_tensor(lhs, rhs)
+            else:
+                return lhs, rhs
+
+        result = iterative_let(let, _each_binding, self.mk_let(dynamic_regions))
+        assert result
+        return result
+
+    def process_alloc_storage(self, dynamic_regions, lhs, call):
+        """Process alloc_storage"""
+        size, alignment = call.args
+        dtype = call.attrs.dtype
+        ctx = TVMContext(call.attrs.device_type, call.attrs.device_id)
+
+        if not isinstance(size, expr.Constant):
+            self.enter_scope()
+            dynamic_regions.append(lhs)
+
+        region = self.current_region(dtype)
+        region.grow(lhs, size, alignment, ctx, dtype)
+        return lhs, region.var
+
+    def process_alloc_tensor(self, lhs, call):
+        """Process alloc tensor. Region and offset are computed"""
+        storage, old_offset, shape = call.args
+        region, offset = self.new_region_and_offset(storage)
+
+        assert (
+            old_offset.data.asnumpy().item() == 0
+        ), "no offsets should yet be allocated"
+        return (
+            lhs,
+            expr.Call(call.op, [region.var, offset, shape], call.attrs),
+        )
+
+class LiftConst(ExprMutator):
+    """A internal pass to lift constants to the top level of function."""
+    def __init__(self):
+        self.i = 0
+        self.constants = []
+        self.top_level = True
+        super().__init__()
+
+    def visit_constant(self, const):
+        var = expr.var(f"const{self.i}")
+        self.i += 1
+        self.constants.append((var, const))
+        return var
+
+    def visit_function(self, fn):
+        if int(getattr(fn.attrs, "Primitive", 0)) == 1:
+            return fn
+
+        outer_constant = self.constants
+        self.constants = []
+        body = mk_let(self.constants, self.visit(fn.body))

Review comment:
       oh this is relying a very subtle side-effect, the visit on the right is populating the variable again. 




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

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



[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #5144:
URL: https://github.com/apache/incubator-tvm/pull/5144#discussion_r422363192



##########
File path: src/runtime/vm/vm.cc
##########
@@ -535,6 +544,7 @@ void InstructionPrint(std::ostream& os, const Instruction& instr) {
     case Opcode::AllocTensorReg: {
       os << "alloc_tensor_reg $" << instr.dst << " $"
          << instr.alloc_tensor_reg.storage << " $"

Review comment:
       ```suggestion
            << instr.alloc_tensor_reg.storage << " "
   ```

##########
File path: src/runtime/vm/vm.cc
##########
@@ -535,6 +544,7 @@ void InstructionPrint(std::ostream& os, const Instruction& instr) {
     case Opcode::AllocTensorReg: {
       os << "alloc_tensor_reg $" << instr.dst << " $"
          << instr.alloc_tensor_reg.storage << " $"
+         << instr.alloc_tensor_reg.offset << " "

Review comment:
       ```suggestion
            << instr.alloc_tensor_reg.offset << " $"
   ```




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

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



[GitHub] [incubator-tvm] icemelon9 merged pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 merged pull request #5144:
URL: https://github.com/apache/incubator-tvm/pull/5144


   


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

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



[GitHub] [incubator-tvm] jroesch commented on pull request #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5144:
URL: https://github.com/apache/incubator-tvm/pull/5144#issuecomment-618651343


   @icemelon9 yes my intention is we can in theory dynamically compute everything like malloc or pool allocators. 


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

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



[GitHub] [incubator-tvm] icemelon9 commented on issue #5144: [Relay][VM] Memory planner (part 1)

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #5144:
URL: https://github.com/apache/incubator-tvm/pull/5144#issuecomment-618059999


   I have a general question. Currently `offset` is a constant. Is it possible that the `offset` is dynamically computed, like the allocation size?


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

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