You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/11/04 23:22:26 UTC

[GitHub] [tvm] mbs-octoml commented on a change in pull request #8468: [TIR][USMP] Added buffer info extraction pass

mbs-octoml commented on a change in pull request #8468:
URL: https://github.com/apache/tvm/pull/8468#discussion_r743248398



##########
File path: include/tvm/tir/usmp/utils.h
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/usmp/utils.h
+ * \brief Utilities for Unified Static Memory Planner
+ */
+
+#ifndef TVM_TIR_USMP_UTILS_H_
+#define TVM_TIR_USMP_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/stmt.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*!
+ * \brief The string parameter to indicate read and write access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_WRITE_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadWriteAccess = "rw";
+/*!
+ * \brief The string parameter to indicate read only access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_ONLY_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadOnlyAccess = "ro";
+
+/*!
+ * \brief The pool information to be used by USMP
+ */
+struct PoolInfoNode : public Object {
+  /*! \brief The name of the memory pool */
+  String pool_name;
+  /*! \brief The expected size hint to be used by the allocator.
+   * The size_hint_bytes is defaulted to kUnrestrictedPoolSizeHint
+   * to indicate the pool is not size restricted.
+   */
+  Integer size_hint_bytes;
+  /*! \brief The accessibility from each Target*/
+  Map<Target, String> target_access;  // 'rw' or 'ro'
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("pool_name", &pool_name);
+    v->Visit("size_hint_bytes", &size_hint_bytes);
+    v->Visit("target_access", &target_access);
+  }
+
+  bool SEqualReduce(const PoolInfoNode* other, SEqualReducer equal) const {
+    return equal(pool_name, other->pool_name) && equal(size_hint_bytes, other->size_hint_bytes) &&
+           equal(target_access, other->target_access);
+  }
+
+  void SHashReduce(SHashReducer hash_reduce) const {
+    hash_reduce(pool_name);
+    hash_reduce(size_hint_bytes);
+    hash_reduce(target_access);
+  }
+
+  static constexpr const char* _type_key = "tir.usmp.PoolInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PoolInfoNode, Object);
+};
+
+/*!
+ * \brief The PoolSize is unrestricted for the memory planner
+ */
+static const int kUnrestrictedPoolSizeHint = -1;
+
+class PoolInfo : public ObjectRef {
+ public:
+  TVM_DLL PoolInfo(String pool_name, Map<Target, String> target_access,
+                   Integer size_hint_bytes = kUnrestrictedPoolSizeHint);
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(PoolInfo, ObjectRef, PoolInfoNode);
+};
+
+/*!
+ * \brief The buffer information to be used by USMP

Review comment:
       nit: "Describes an abstract memory buffer inside a pool. The actual memory buffer in represented by PoolAllocationNode after static memory planning.'

##########
File path: include/tvm/tir/usmp/utils.h
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/usmp/utils.h
+ * \brief Utilities for Unified Static Memory Planner
+ */
+
+#ifndef TVM_TIR_USMP_UTILS_H_
+#define TVM_TIR_USMP_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/stmt.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*!
+ * \brief The string parameter to indicate read and write access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_WRITE_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadWriteAccess = "rw";
+/*!
+ * \brief The string parameter to indicate read only access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_ONLY_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadOnlyAccess = "ro";
+
+/*!
+ * \brief The pool information to be used by USMP
+ */
+struct PoolInfoNode : public Object {
+  /*! \brief The name of the memory pool */
+  String pool_name;
+  /*! \brief The expected size hint to be used by the allocator.
+   * The size_hint_bytes is defaulted to kUnrestrictedPoolSizeHint
+   * to indicate the pool is not size restricted.
+   */
+  Integer size_hint_bytes;
+  /*! \brief The accessibility from each Target*/
+  Map<Target, String> target_access;  // 'rw' or 'ro'
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("pool_name", &pool_name);
+    v->Visit("size_hint_bytes", &size_hint_bytes);
+    v->Visit("target_access", &target_access);
+  }
+
+  bool SEqualReduce(const PoolInfoNode* other, SEqualReducer equal) const {
+    return equal(pool_name, other->pool_name) && equal(size_hint_bytes, other->size_hint_bytes) &&
+           equal(target_access, other->target_access);
+  }
+
+  void SHashReduce(SHashReducer hash_reduce) const {
+    hash_reduce(pool_name);
+    hash_reduce(size_hint_bytes);
+    hash_reduce(target_access);
+  }
+
+  static constexpr const char* _type_key = "tir.usmp.PoolInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PoolInfoNode, Object);
+};
+
+/*!
+ * \brief The PoolSize is unrestricted for the memory planner
+ */
+static const int kUnrestrictedPoolSizeHint = -1;
+
+class PoolInfo : public ObjectRef {
+ public:
+  TVM_DLL PoolInfo(String pool_name, Map<Target, String> target_access,
+                   Integer size_hint_bytes = kUnrestrictedPoolSizeHint);
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(PoolInfo, ObjectRef, PoolInfoNode);
+};
+
+/*!
+ * \brief The buffer information to be used by USMP
+ */
+struct BufferInfoNode : public Object {
+  /*! \brief The name of the buffer var */
+  String name_hint;
+  /*! \brief The size in terms of bytes */
+  Integer size_bytes;
+  /*! \brief The pool candidates that this buffer can get pooled to*/
+  Array<PoolInfo> pool_candidates;
+  /*! \brief The byte alignment required within the pool */
+  Integer alignment;

Review comment:
       nit: I guess this has to be the gcm of the alignment constraints imposed by all user's of buffer -- is there any use for an Array<Target> to capture that?

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+
+class BufferInfoExtractor : public StmtExprVisitor {
+ public:
+  explicit BufferInfoExtractor(const IRModule& module) : module_(module) {
+    for (const auto& gv_func : module_->functions) {
+      functions_.Set(gv_func.first->name_hint, Downcast<PrimFunc>(gv_func.second));
+    }
+    // Pushing a scope info for the initial body of the main function
+    scope_stack_.push(ScopeInfo());
+  }
+  Map<BufferInfo, tir::Stmt> operator()(const PrimFunc& func);
+
+ private:
+  void VisitStmt(const Stmt& n) override;
+  void VisitStmt_(const AllocateNode* op) override;
+  void VisitExpr_(const CallNode* op) override;
+  void VisitExpr_(const VarNode* op) override;
+  void VisitExpr_(const LoadNode* op) override;
+  void VisitStmt_(const StoreNode* op) override;
+  void VisitStmt_(const ForNode* op) override;
+
+  void UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func);
+
+  Map<BufferInfo, tir::Stmt> buffer_info_map_;
+  Map<tir::Stmt, Integer> buffer_info_start_stmt_idx_;
+  Map<tir::Stmt, Integer> buffer_info_end_stmt_idx_;
+  Map<tir::Var, tir::Stmt> allocate_var_to_stmt_map_;
+
+  std::unordered_set<Stmt, ObjectPtrHash, ObjectPtrEqual> currently_live_allocates;
+  int current_stmt_idx_ = 0;
+  // This structure is supposed to contain information
+  // around the scope the visitor is currently in.
+  // We only check whether the current scope belong to
+  // a Serial ForKind. We are not planning for Parallel
+  // ForKind just yet.
+  struct ScopeInfo {
+    For for_loop;
+  };
+  std::stack<ScopeInfo> scope_stack_;
+
+  Map<String, PrimFunc> functions_;
+  IRModule module_;
+};
+
+void BufferInfoExtractor::VisitStmt(const Stmt& n) {
+  current_stmt_idx_ += 1;
+  StmtExprVisitor::VisitStmt(n);
+}
+
+static Integer CalculateExtentsSize(const AllocateNode* op) {
+  size_t element_size_bytes = op->dtype.bytes();
+  size_t num_elements = 1;
+  for (const auto& ext : op->extents) {
+    if (ext->IsInstance<IntImmNode>()) {
+      num_elements *= Downcast<IntImm>(ext)->value;
+    } else {
+      // We can't statically calculate workspace for dynamic shapes
+      return Integer();
+    }
+  }
+  return Integer(num_elements * element_size_bytes);
+}
+
+void BufferInfoExtractor::VisitStmt_(const AllocateNode* op) {
+  const auto& currect_scope_info = scope_stack_.top();
+  const auto& type = Downcast<PointerType>(op->buffer_var->type_annotation);
+  const auto& storage_scope = type->storage_scope;
+
+  // If the allocate is in a for loop, USMP currently only looks at serial for loops.
+  // If its not a serial for loop, then memory planner will omit them in the current memory planning
+  // process leaving them to as tir.allocate nodes for codegen. Additionally, the USMP can only work
+  // with buffers that have global storage_scope
+  if ((!currect_scope_info.for_loop.defined()) ||
+      (currect_scope_info.for_loop.defined() &&
+       currect_scope_info.for_loop->kind == ForKind::kSerial && storage_scope == "global")) {
+    auto size_bytes = CalculateExtentsSize(op);
+    // We only statically memory plan only allocates with known
+    // compile time sizes.
+    if (size_bytes.defined()) {
+      // By default, the core compiler is assumed to attach the a default pool to each allocate.
+      ICHECK(op->annotations.count(kPoolCandidatesAllocateAttr))
+          << "Every statically sized allocate node needs an pool candidate attribute";
+      auto pool_candidates =
+          Downcast<Array<PoolInfo>>(op->annotations[kPoolCandidatesAllocateAttr]);
+
+      // TODO(@manupa-arm): improve the error when the responsible component for attaching a single
+      // pool is added
+      ICHECK(pool_candidates.size() > 0)
+          << "The core compiler should at least attach a single PoolInfo. If there were no "
+             "user-given arguments for memory pools, the default behaviour is a single size "
+             "un-restricted pool is assigned";
+      auto buffer_info = BufferInfo(op->buffer_var->name_hint, size_bytes, pool_candidates);
+      auto allocate = GetRef<Allocate>(op);
+      allocate_var_to_stmt_map_.Set(op->buffer_var, allocate);
+      buffer_info_map_.Set(buffer_info, allocate);
+    }
+  }
+  StmtExprVisitor::VisitStmt(op->body);
+}
+
+void BufferInfoExtractor::VisitStmt_(const ForNode* op) {
+  ScopeInfo si{
+      GetRef<For>(op),
+  };
+  scope_stack_.push(si);
+  StmtExprVisitor::VisitStmt_(op);
+  scope_stack_.pop();
+}
+
+void BufferInfoExtractor::VisitExpr_(const LoadNode* op) {
+  this->VisitExpr(op->buffer_var);
+  StmtExprVisitor::VisitExpr_(op);
+}
+
+void BufferInfoExtractor::VisitStmt_(const StoreNode* op) {
+  this->VisitExpr(op->buffer_var);
+  StmtExprVisitor::VisitStmt_(op);
+}
+
+void BufferInfoExtractor::VisitExpr_(const VarNode* op) {
+  auto var = GetRef<Var>(op);
+  if (allocate_var_to_stmt_map_.count(var)) {
+    auto allocate = allocate_var_to_stmt_map_[var];
+    if (buffer_info_start_stmt_idx_.count(allocate) == 0) {
+      buffer_info_start_stmt_idx_.Set(allocate, current_stmt_idx_);
+    }
+    buffer_info_end_stmt_idx_.Set(allocate, current_stmt_idx_);
+  }
+  StmtExprVisitor::VisitExpr_(op);
+}
+
+Array<Var> static GetMatchedBuffers(const PrimFunc& func) {
+  Array<Var> buffer_vars;
+  for (const auto& param : func->params) {
+    buffer_vars.push_back(func->buffer_map[param]->data);
+  }
+  return buffer_vars;
+}
+
+void BufferInfoExtractor::UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func) {

Review comment:
       At most one call per function, right? Or at least if there are multiple calls then they all use the same buffers. Am I reading that right? CHECK fail if that assumption fails?

##########
File path: python/tvm/tir/usmp/analysis/analysis.py
##########
@@ -0,0 +1,39 @@
+# 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.
+"""USMP Analysis Python API for passes"""
+# pylint: disable=invalid-name
+from . import _ffi_api
+from ...function import PrimFunc
+from ....ir.module import IRModule
+
+
+def extract_buffer_info(main_func: PrimFunc, mod: IRModule):
+    """Convert Parallel For Loop to Serial.

Review comment:
       copy pasta

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+

Review comment:
       super-nit: no linebreak. (Yeah, ocd, I know.)

##########
File path: include/tvm/tir/usmp/utils.h
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/usmp/utils.h
+ * \brief Utilities for Unified Static Memory Planner
+ */
+
+#ifndef TVM_TIR_USMP_UTILS_H_
+#define TVM_TIR_USMP_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/stmt.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*!
+ * \brief The string parameter to indicate read and write access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_WRITE_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadWriteAccess = "rw";
+/*!
+ * \brief The string parameter to indicate read only access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_ONLY_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadOnlyAccess = "ro";
+
+/*!
+ * \brief The pool information to be used by USMP

Review comment:
       nit: "Describes a pool of memory accessible by one or more targets."

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo

Review comment:
       nit: callees

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+
+class BufferInfoExtractor : public StmtExprVisitor {
+ public:
+  explicit BufferInfoExtractor(const IRModule& module) : module_(module) {
+    for (const auto& gv_func : module_->functions) {
+      functions_.Set(gv_func.first->name_hint, Downcast<PrimFunc>(gv_func.second));
+    }
+    // Pushing a scope info for the initial body of the main function
+    scope_stack_.push(ScopeInfo());
+  }
+  Map<BufferInfo, tir::Stmt> operator()(const PrimFunc& func);
+
+ private:
+  void VisitStmt(const Stmt& n) override;
+  void VisitStmt_(const AllocateNode* op) override;
+  void VisitExpr_(const CallNode* op) override;
+  void VisitExpr_(const VarNode* op) override;
+  void VisitExpr_(const LoadNode* op) override;
+  void VisitStmt_(const StoreNode* op) override;
+  void VisitStmt_(const ForNode* op) override;
+
+  void UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func);
+
+  Map<BufferInfo, tir::Stmt> buffer_info_map_;

Review comment:
       comments for all these fields please

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+

Review comment:
       A quick overview of the alg would be great: collect the min/max statement indexes for every buffer var occurrence, the BufferInfos for each tir.allocate, and the (unique, right?) BufferInfos for func params  from calls, then sweep the min/max intervals for the conflicts.
   
   There's no control flow to worry about so simple min/max regions suffice.
   

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo

Review comment:
       actually no need to dup the class comment -- just a one liner here seems fine

##########
File path: include/tvm/tir/usmp/utils.h
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/usmp/utils.h
+ * \brief Utilities for Unified Static Memory Planner
+ */
+
+#ifndef TVM_TIR_USMP_UTILS_H_
+#define TVM_TIR_USMP_UTILS_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/stmt.h>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*!
+ * \brief The string parameter to indicate read and write access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_WRITE_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadWriteAccess = "rw";
+/*!
+ * \brief The string parameter to indicate read only access to a pool
+ * This needs to be kept in sync with PoolInfo.READ_ONLY_ACCESS in
+ * python/tvm/tir/usmp/utils.py
+ */
+static constexpr const char* kTargetPoolReadOnlyAccess = "ro";
+
+/*!
+ * \brief The pool information to be used by USMP
+ */
+struct PoolInfoNode : public Object {
+  /*! \brief The name of the memory pool */
+  String pool_name;
+  /*! \brief The expected size hint to be used by the allocator.
+   * The size_hint_bytes is defaulted to kUnrestrictedPoolSizeHint
+   * to indicate the pool is not size restricted.
+   */
+  Integer size_hint_bytes;
+  /*! \brief The accessibility from each Target*/
+  Map<Target, String> target_access;  // 'rw' or 'ro'
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("pool_name", &pool_name);
+    v->Visit("size_hint_bytes", &size_hint_bytes);
+    v->Visit("target_access", &target_access);
+  }
+
+  bool SEqualReduce(const PoolInfoNode* other, SEqualReducer equal) const {
+    return equal(pool_name, other->pool_name) && equal(size_hint_bytes, other->size_hint_bytes) &&
+           equal(target_access, other->target_access);
+  }
+
+  void SHashReduce(SHashReducer hash_reduce) const {
+    hash_reduce(pool_name);
+    hash_reduce(size_hint_bytes);
+    hash_reduce(target_access);
+  }
+
+  static constexpr const char* _type_key = "tir.usmp.PoolInfo";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PoolInfoNode, Object);
+};
+
+/*!
+ * \brief The PoolSize is unrestricted for the memory planner
+ */
+static const int kUnrestrictedPoolSizeHint = -1;
+
+class PoolInfo : public ObjectRef {
+ public:
+  TVM_DLL PoolInfo(String pool_name, Map<Target, String> target_access,
+                   Integer size_hint_bytes = kUnrestrictedPoolSizeHint);
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(PoolInfo, ObjectRef, PoolInfoNode);
+};
+
+/*!
+ * \brief The buffer information to be used by USMP

Review comment:
       can you do me a favor  and add 'see alsos' for:
      relay::StorageToken (graph_plan_memory.cc)
      relay::backend::StorageInfoNdoe (relay/backend/utils.h)
      Region (python/tvm/relay/transform/memory_plan.py)

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+
+class BufferInfoExtractor : public StmtExprVisitor {
+ public:
+  explicit BufferInfoExtractor(const IRModule& module) : module_(module) {
+    for (const auto& gv_func : module_->functions) {
+      functions_.Set(gv_func.first->name_hint, Downcast<PrimFunc>(gv_func.second));
+    }
+    // Pushing a scope info for the initial body of the main function
+    scope_stack_.push(ScopeInfo());
+  }
+  Map<BufferInfo, tir::Stmt> operator()(const PrimFunc& func);
+
+ private:
+  void VisitStmt(const Stmt& n) override;
+  void VisitStmt_(const AllocateNode* op) override;
+  void VisitExpr_(const CallNode* op) override;
+  void VisitExpr_(const VarNode* op) override;
+  void VisitExpr_(const LoadNode* op) override;
+  void VisitStmt_(const StoreNode* op) override;
+  void VisitStmt_(const ForNode* op) override;
+
+  void UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func);
+
+  Map<BufferInfo, tir::Stmt> buffer_info_map_;
+  Map<tir::Stmt, Integer> buffer_info_start_stmt_idx_;
+  Map<tir::Stmt, Integer> buffer_info_end_stmt_idx_;
+  Map<tir::Var, tir::Stmt> allocate_var_to_stmt_map_;
+
+  std::unordered_set<Stmt, ObjectPtrHash, ObjectPtrEqual> currently_live_allocates;
+  int current_stmt_idx_ = 0;
+  // This structure is supposed to contain information
+  // around the scope the visitor is currently in.
+  // We only check whether the current scope belong to
+  // a Serial ForKind. We are not planning for Parallel
+  // ForKind just yet.
+  struct ScopeInfo {
+    For for_loop;
+  };
+  std::stack<ScopeInfo> scope_stack_;
+
+  Map<String, PrimFunc> functions_;
+  IRModule module_;
+};
+
+void BufferInfoExtractor::VisitStmt(const Stmt& n) {
+  current_stmt_idx_ += 1;
+  StmtExprVisitor::VisitStmt(n);
+}
+
+static Integer CalculateExtentsSize(const AllocateNode* op) {

Review comment:
       comments for all the non-overridden methods please

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+
+class BufferInfoExtractor : public StmtExprVisitor {
+ public:
+  explicit BufferInfoExtractor(const IRModule& module) : module_(module) {
+    for (const auto& gv_func : module_->functions) {
+      functions_.Set(gv_func.first->name_hint, Downcast<PrimFunc>(gv_func.second));
+    }
+    // Pushing a scope info for the initial body of the main function
+    scope_stack_.push(ScopeInfo());
+  }
+  Map<BufferInfo, tir::Stmt> operator()(const PrimFunc& func);
+
+ private:
+  void VisitStmt(const Stmt& n) override;
+  void VisitStmt_(const AllocateNode* op) override;
+  void VisitExpr_(const CallNode* op) override;
+  void VisitExpr_(const VarNode* op) override;
+  void VisitExpr_(const LoadNode* op) override;
+  void VisitStmt_(const StoreNode* op) override;
+  void VisitStmt_(const ForNode* op) override;
+
+  void UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func);
+
+  Map<BufferInfo, tir::Stmt> buffer_info_map_;
+  Map<tir::Stmt, Integer> buffer_info_start_stmt_idx_;
+  Map<tir::Stmt, Integer> buffer_info_end_stmt_idx_;
+  Map<tir::Var, tir::Stmt> allocate_var_to_stmt_map_;
+
+  std::unordered_set<Stmt, ObjectPtrHash, ObjectPtrEqual> currently_live_allocates;
+  int current_stmt_idx_ = 0;
+  // This structure is supposed to contain information
+  // around the scope the visitor is currently in.
+  // We only check whether the current scope belong to
+  // a Serial ForKind. We are not planning for Parallel
+  // ForKind just yet.
+  struct ScopeInfo {
+    For for_loop;
+  };
+  std::stack<ScopeInfo> scope_stack_;
+
+  Map<String, PrimFunc> functions_;
+  IRModule module_;
+};
+
+void BufferInfoExtractor::VisitStmt(const Stmt& n) {
+  current_stmt_idx_ += 1;
+  StmtExprVisitor::VisitStmt(n);
+}
+
+static Integer CalculateExtentsSize(const AllocateNode* op) {
+  size_t element_size_bytes = op->dtype.bytes();
+  size_t num_elements = 1;
+  for (const auto& ext : op->extents) {
+    if (ext->IsInstance<IntImmNode>()) {
+      num_elements *= Downcast<IntImm>(ext)->value;
+    } else {
+      // We can't statically calculate workspace for dynamic shapes
+      return Integer();
+    }
+  }
+  return Integer(num_elements * element_size_bytes);
+}
+
+void BufferInfoExtractor::VisitStmt_(const AllocateNode* op) {
+  const auto& currect_scope_info = scope_stack_.top();

Review comment:
       nit: current

##########
File path: src/tir/usmp/analysis/extract_buffer_info.cc
##########
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/extract_buffer_info.cc
+ *
+ * \brief This analysis pass consumes a TIR IRModule with a main function
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <stack>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+
+/*! \brief This class takes a TIR IRModule and a main PrimFunc that contains
+ * that defines a ordering in the calles to operators and produces BufferInfo
+ * objects that contains information about tir.allocate nodes and liveness
+ * conflicts between other tir.allocate nodes.
+ */
+
+class BufferInfoExtractor : public StmtExprVisitor {
+ public:
+  explicit BufferInfoExtractor(const IRModule& module) : module_(module) {
+    for (const auto& gv_func : module_->functions) {
+      functions_.Set(gv_func.first->name_hint, Downcast<PrimFunc>(gv_func.second));
+    }
+    // Pushing a scope info for the initial body of the main function
+    scope_stack_.push(ScopeInfo());
+  }
+  Map<BufferInfo, tir::Stmt> operator()(const PrimFunc& func);
+
+ private:
+  void VisitStmt(const Stmt& n) override;
+  void VisitStmt_(const AllocateNode* op) override;
+  void VisitExpr_(const CallNode* op) override;
+  void VisitExpr_(const VarNode* op) override;
+  void VisitExpr_(const LoadNode* op) override;
+  void VisitStmt_(const StoreNode* op) override;
+  void VisitStmt_(const ForNode* op) override;
+
+  void UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func);
+
+  Map<BufferInfo, tir::Stmt> buffer_info_map_;
+  Map<tir::Stmt, Integer> buffer_info_start_stmt_idx_;
+  Map<tir::Stmt, Integer> buffer_info_end_stmt_idx_;
+  Map<tir::Var, tir::Stmt> allocate_var_to_stmt_map_;
+
+  std::unordered_set<Stmt, ObjectPtrHash, ObjectPtrEqual> currently_live_allocates;
+  int current_stmt_idx_ = 0;
+  // This structure is supposed to contain information
+  // around the scope the visitor is currently in.
+  // We only check whether the current scope belong to
+  // a Serial ForKind. We are not planning for Parallel
+  // ForKind just yet.
+  struct ScopeInfo {
+    For for_loop;
+  };
+  std::stack<ScopeInfo> scope_stack_;
+
+  Map<String, PrimFunc> functions_;
+  IRModule module_;
+};
+
+void BufferInfoExtractor::VisitStmt(const Stmt& n) {
+  current_stmt_idx_ += 1;
+  StmtExprVisitor::VisitStmt(n);
+}
+
+static Integer CalculateExtentsSize(const AllocateNode* op) {
+  size_t element_size_bytes = op->dtype.bytes();
+  size_t num_elements = 1;
+  for (const auto& ext : op->extents) {
+    if (ext->IsInstance<IntImmNode>()) {
+      num_elements *= Downcast<IntImm>(ext)->value;
+    } else {
+      // We can't statically calculate workspace for dynamic shapes
+      return Integer();
+    }
+  }
+  return Integer(num_elements * element_size_bytes);
+}
+
+void BufferInfoExtractor::VisitStmt_(const AllocateNode* op) {
+  const auto& currect_scope_info = scope_stack_.top();
+  const auto& type = Downcast<PointerType>(op->buffer_var->type_annotation);
+  const auto& storage_scope = type->storage_scope;
+
+  // If the allocate is in a for loop, USMP currently only looks at serial for loops.
+  // If its not a serial for loop, then memory planner will omit them in the current memory planning
+  // process leaving them to as tir.allocate nodes for codegen. Additionally, the USMP can only work
+  // with buffers that have global storage_scope
+  if ((!currect_scope_info.for_loop.defined()) ||
+      (currect_scope_info.for_loop.defined() &&
+       currect_scope_info.for_loop->kind == ForKind::kSerial && storage_scope == "global")) {
+    auto size_bytes = CalculateExtentsSize(op);
+    // We only statically memory plan only allocates with known
+    // compile time sizes.
+    if (size_bytes.defined()) {
+      // By default, the core compiler is assumed to attach the a default pool to each allocate.
+      ICHECK(op->annotations.count(kPoolCandidatesAllocateAttr))
+          << "Every statically sized allocate node needs an pool candidate attribute";
+      auto pool_candidates =
+          Downcast<Array<PoolInfo>>(op->annotations[kPoolCandidatesAllocateAttr]);
+
+      // TODO(@manupa-arm): improve the error when the responsible component for attaching a single
+      // pool is added
+      ICHECK(pool_candidates.size() > 0)
+          << "The core compiler should at least attach a single PoolInfo. If there were no "
+             "user-given arguments for memory pools, the default behaviour is a single size "
+             "un-restricted pool is assigned";
+      auto buffer_info = BufferInfo(op->buffer_var->name_hint, size_bytes, pool_candidates);
+      auto allocate = GetRef<Allocate>(op);
+      allocate_var_to_stmt_map_.Set(op->buffer_var, allocate);
+      buffer_info_map_.Set(buffer_info, allocate);
+    }
+  }
+  StmtExprVisitor::VisitStmt(op->body);
+}
+
+void BufferInfoExtractor::VisitStmt_(const ForNode* op) {
+  ScopeInfo si{
+      GetRef<For>(op),
+  };
+  scope_stack_.push(si);
+  StmtExprVisitor::VisitStmt_(op);
+  scope_stack_.pop();
+}
+
+void BufferInfoExtractor::VisitExpr_(const LoadNode* op) {
+  this->VisitExpr(op->buffer_var);
+  StmtExprVisitor::VisitExpr_(op);
+}
+
+void BufferInfoExtractor::VisitStmt_(const StoreNode* op) {
+  this->VisitExpr(op->buffer_var);
+  StmtExprVisitor::VisitStmt_(op);
+}
+
+void BufferInfoExtractor::VisitExpr_(const VarNode* op) {
+  auto var = GetRef<Var>(op);
+  if (allocate_var_to_stmt_map_.count(var)) {
+    auto allocate = allocate_var_to_stmt_map_[var];
+    if (buffer_info_start_stmt_idx_.count(allocate) == 0) {
+      buffer_info_start_stmt_idx_.Set(allocate, current_stmt_idx_);
+    }
+    buffer_info_end_stmt_idx_.Set(allocate, current_stmt_idx_);
+  }
+  StmtExprVisitor::VisitExpr_(op);
+}
+
+Array<Var> static GetMatchedBuffers(const PrimFunc& func) {
+  Array<Var> buffer_vars;
+  for (const auto& param : func->params) {
+    buffer_vars.push_back(func->buffer_map[param]->data);
+  }
+  return buffer_vars;
+}
+
+void BufferInfoExtractor::UpdateAliases(const Array<PrimExpr>& args, const PrimFunc& func) {

Review comment:
       this also assumes the calls are seen after the PrimFunc defns, 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.

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

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