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/12/13 15:58:19 UTC

[GitHub] [tvm] manupa-arm commented on a change in pull request #9704: [USMP] Hill Climb allocator

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



##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc

Review comment:
       Should we clarify what is the modification ?

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;

Review comment:
       I think we dont need to define a new function _ptr_hash here. Please take look at the following example.
   
   https://github.com/apache/tvm/blob/5557b8c4eed73fcbef60269bb8e0eead35feeee0/src/tir/usmp/algo/greedy.cc#L93

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rand_r(&_seedp) % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;
+      total_size = total;
+
+      // reached desired size
+      if (total_size <= desired_bytes_) {
+        break;
+      }
+
+    } else {
+      // rollback
+      swap_buffers(swap_i2, swap_i1);
+      pool_allocations = std::move(rollback_pool_allocations);
+      max_pool_size = find_highest(&pool_allocations);
+    }
+
+    std::vector<const BufferInfoNode*> max_pool_buf;
+
+    for (const auto& it : pool_allocations) {
+      const auto* buf = it.first;
+      const auto pa = it.second;
+      size_t high_sz = pa->byte_offset + buf->size_bytes;
+      if (max_pool_size[pa->pool_info] == high_sz) {
+        max_pool_buf.push_back(buf);
+      }
+    }
+
+    // pick highest
+    const BufferInfoNode* suspect = max_pool_buf[rand() % max_pool_buf.size()];
+    PoolAllocation suspect_pa = pool_allocations[suspect];
+
+    std::unordered_map<int, const BufferInfoNode*, _ptr_hash> first_level_set;
+    std::unordered_map<int, const BufferInfoNode*, _ptr_hash> second_level_set;

Review comment:
       Same here

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rnd_func() % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;

Review comment:
       I cant seem to find any further uses of this. Would be able to point me to if any ?

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):
+    random.seed(0)
+    mem_range = [BufferInfo(str(i), random.randrange(1, 65535), pools) for i in range(random_len)]
+    for mr in mem_range:
+        pr = random.choice(mem_range)
+        while pr in (*mr.conflicts, mr):
+            pr = random.choice(mem_range)
+
+        mr.set_conflicts([*mr.conflicts, pr])
+        pr.set_conflicts([*pr.conflicts, mr])
+
+    fusmp_algo = tvm.get_global_func("tir.usmp.algo.hill_climb")
+    result_map = fusmp_algo(mem_range, 0)
+    _verify_all_conflicts(result_map)
+
+
+def __test_data_alloc_max():
+    intervals = [
+        (0, 159, 2048),
+        (0, 13, 7904),
+        (4, 35, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+    ]
+    return intervals
+
+
+def __test_data_deep_speech():
+    intervals = [
+        (0, 159, 2048),
+        (0, 151, 2048),
+        (0, 13, 7904),
+        (2, 49, 16),
+        (4, 35, 16),
+        (6, 21, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+        (20, 27, 32768),
+        (26, 31, 32768),
+        (30, 35, 32768),
+        (34, 41, 32768),
+        (40, 45, 32768),
+        (44, 49, 32768),
+        (48, 145, 32768),
+        (54, 59, 2048),
+        (58, 483, 4096),
+        (60, 65, 2048),
+        (64, 461, 4096),
+        (66, 71, 2048),
+        (70, 439, 4096),
+        (72, 77, 2048),
+        (76, 417, 4096),
+        (78, 83, 2048),
+        (82, 395, 4096),
+        (84, 89, 2048),
+        (88, 373, 4096),
+        (90, 95, 2048),
+        (94, 351, 4096),
+        (96, 101, 2048),
+        (100, 329, 4096),
+        (102, 107, 2048),
+        (106, 307, 4096),
+        (108, 113, 2048),
+        (112, 285, 4096),
+        (114, 119, 2048),
+        (118, 263, 4096),
+        (120, 125, 2048),
+        (124, 241, 4096),
+        (126, 131, 2048),
+        (130, 219, 4096),
+        (132, 137, 2048),
+        (136, 197, 4096),
+        (138, 143, 2048),
+        (142, 175, 4096),
+        (144, 149, 2048),
+        (148, 153, 4096),
+        (152, 163, 8192),
+        (154, 171, 2048),
+        (156, 181, 2048),
+        (160, 167, 2048),
+        (162, 165, 2048),
+        (168, 171, 2048),
+        (170, 509, 2048),
+        (174, 185, 8192),
+        (176, 193, 2048),
+        (178, 203, 2048),
+        (182, 189, 2048),
+        (184, 187, 2048),
+        (190, 193, 2048),
+        (192, 511, 2048),
+        (196, 207, 8192),
+        (198, 215, 2048),
+        (200, 225, 2048),
+        (204, 211, 2048),
+        (206, 209, 2048),
+        (212, 215, 2048),
+        (214, 513, 2048),
+        (218, 229, 8192),
+        (220, 237, 2048),
+        (222, 247, 2048),
+        (226, 233, 2048),
+        (228, 231, 2048),
+        (234, 237, 2048),
+        (236, 515, 2048),
+        (240, 251, 8192),
+        (242, 259, 2048),
+        (244, 269, 2048),
+        (248, 255, 2048),
+        (250, 253, 2048),
+        (256, 259, 2048),
+        (258, 517, 2048),
+        (262, 273, 8192),
+        (264, 281, 2048),
+        (266, 291, 2048),
+        (270, 277, 2048),
+        (272, 275, 2048),
+        (278, 281, 2048),
+        (280, 519, 2048),
+        (284, 295, 8192),
+        (286, 303, 2048),
+        (288, 313, 2048),
+        (292, 299, 2048),
+        (294, 297, 2048),
+        (300, 303, 2048),
+        (302, 521, 2048),
+        (306, 317, 8192),
+        (308, 325, 2048),
+        (310, 335, 2048),
+        (314, 321, 2048),
+        (316, 319, 2048),
+        (322, 325, 2048),
+        (324, 523, 2048),
+        (328, 339, 8192),
+        (330, 347, 2048),
+        (332, 357, 2048),
+        (336, 343, 2048),
+        (338, 341, 2048),
+        (344, 347, 2048),
+        (346, 525, 2048),
+        (350, 361, 8192),
+        (352, 369, 2048),
+        (354, 379, 2048),
+        (358, 365, 2048),
+        (360, 363, 2048),
+        (366, 369, 2048),
+        (368, 527, 2048),
+        (372, 383, 8192),
+        (374, 391, 2048),
+        (376, 401, 2048),
+        (380, 387, 2048),
+        (382, 385, 2048),
+        (388, 391, 2048),
+        (390, 529, 2048),
+        (394, 405, 8192),
+        (396, 413, 2048),
+        (398, 423, 2048),
+        (402, 409, 2048),
+        (404, 407, 2048),
+        (410, 413, 2048),
+        (412, 531, 2048),
+        (416, 427, 8192),
+        (418, 435, 2048),
+        (420, 445, 2048),
+        (424, 431, 2048),
+        (426, 429, 2048),
+        (432, 435, 2048),
+        (434, 533, 2048),
+        (438, 449, 8192),
+        (440, 457, 2048),
+        (442, 467, 2048),
+        (446, 453, 2048),
+        (448, 451, 2048),
+        (454, 457, 2048),
+        (456, 535, 2048),
+        (460, 471, 8192),
+        (462, 479, 2048),
+        (464, 489, 2048),
+        (468, 475, 2048),
+        (470, 473, 2048),
+        (476, 479, 2048),
+        (478, 537, 2048),
+        (482, 493, 8192),
+        (484, 501, 2048),
+        (486, 497, 2048),
+        (490, 497, 2048),
+        (492, 495, 2048),
+        (496, 626, 2048),
+        (498, 501, 2048),
+        (500, 626, 2048),
+        (504, 549, 16),
+        (508, 543, 32768),
+        (542, 549, 32768),
+        (548, 555, 32768),
+        (554, 563, 464),
+        (560, 563, 256),
+        (562, 617, 2048),
+        (564, 567, 1856),
+        (566, 573, 1024),
+        (568, 619, 1024),
+        (570, 573, 1024),
+        (572, 577, 1024),
+        (576, 579, 1024),
+        (578, 605, 1024),
+        (580, 593, 1024),
+        (584, 587, 1024),
+        (586, 603, 1024),
+        (594, 597, 1024),
+        (596, 613, 1024),
+        (604, 607, 1024),
+        (606, 617, 1024),
+        (616, 621, 2048),
+        (618, 621, 1024),
+        (620, 626, 464),
+    ]
+    return intervals
+
+
+def __test_data_five():
+    return [
+        (4, 5, 95),
+        (1, 4, 52135),
+        (3, 4, 12136),
+        (3, 5, 62099),
+        (4, 5, 50458),
+    ]
+
+
+def __test_data_simple():
+    return [
+        (0, 23, 131072),  # 0
+        (4, 5, 65568),  # 1
+        (4, 9, 8192),  # 2
+        (8, 30, 15360),  # 3
+        (10, 11, 65568),  # 4
+        (10, 15, 4096),  # 5
+        (16, 17, 65552),  # 6
+        (16, 21, 2048),  # 7
+        (22, 23, 32784),  # 8
+        (22, 27, 1024),  # 9
+    ]
+
+
+def maximumFromIntervals(intervals):
+    # expected list of intervals of (start, end, size)
+    sorted_list = sorted(intervals, key=lambda _: _[0])
+    max_mem = 0
+    for t in range(sorted_list[0][0], sorted_list[-1][1] + 1):
+        max_mem = max(
+            max_mem, sum([size for (start, end, size) in sorted_list if t >= start and t <= end])
+        )
+    return max_mem
+
+
+@pytest.mark.parametrize(
+    "intervals",
+    [__test_data_alloc_max(), __test_data_simple(), __test_data_deep_speech(), __test_data_five()],
+)
+def test_intervals(intervals):
+    result = run_intervals(intervals)
+    assert result["tir.usmp.algo.hill_climb"] == True, f" {result}"
+
+
+def generate_range(sz, max_segment_sz=65535):
+    for i in range(0, sz):
+        start = random.randrange(i, sz)
+        stop = random.randrange(start + 1, start + 2 + ((sz - start) // 2))
+        assert stop - start > 0
+        yield (start, stop, random.randrange(1, max_segment_sz))
+
+
+@pytest.mark.skip()
+def test_10_random_intervals():

Review comment:
       Do we need these anymore ?

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);

Review comment:
       nit : you pass in a reference too.

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;

Review comment:
       remove

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {

Review comment:
       I think we can re-use the GreedyBase class from greedy.cc.
   Feel free to create the header file in this PR for re-using GreedyBase class (simliar to https://github.com/apache/tvm/blob/8cbe0858f0a90351c2564aff2d9c858a2625cdda/include/tvm/tir/usmp/algo/algo.h)

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool

Review comment:
       typo : highest ?

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";

Review comment:
       Shall we improve the error message here a bit ?

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rnd_func() % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;
+      total_size = total;
+
+      // reached desired size
+      if (total_size <= desired_bytes_) {
+        break;
+      }
+
+    } else {
+      // rollback
+      swap_buffers(swap_i2, swap_i1);
+      pool_allocations = std::move(rollback_pool_allocations);
+      max_pool_size = find_highest(&pool_allocations);
+    }
+
+    std::vector<const BufferInfoNode*> max_pool_buf;
+
+    for (const auto& it : pool_allocations) {
+      const auto* buf = it.first;
+      const auto pa = it.second;
+      size_t high_sz = pa->byte_offset + buf->size_bytes;
+      if (max_pool_size[pa->pool_info] == high_sz) {
+        max_pool_buf.push_back(buf);
+      }
+    }
+
+    // pick highest
+    const BufferInfoNode* suspect = max_pool_buf[rand() % max_pool_buf.size()];
+    PoolAllocation suspect_pa = pool_allocations[suspect];
+
+    std::unordered_map<int, const BufferInfoNode*, _ptr_hash> first_level_set;

Review comment:
       We need to provide better documentation on what is meant by first level vs second level.

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(

Review comment:
       I think function will be useful for many algorithm, hence I would suggest to move this utils.cc and expose it via utils.h.
   We should include a brief documenting the function behaviour.

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rand_r(&_seedp) % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {

Review comment:
       We would need to break this line down and probably comment what this condition means.

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rand_r(&_seedp) % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;
+      total_size = total;
+
+      // reached desired size
+      if (total_size <= desired_bytes_) {
+        break;
+      }
+
+    } else {
+      // rollback
+      swap_buffers(swap_i2, swap_i1);
+      pool_allocations = std::move(rollback_pool_allocations);
+      max_pool_size = find_highest(&pool_allocations);
+    }
+
+    std::vector<const BufferInfoNode*> max_pool_buf;
+
+    for (const auto& it : pool_allocations) {
+      const auto* buf = it.first;
+      const auto pa = it.second;
+      size_t high_sz = pa->byte_offset + buf->size_bytes;
+      if (max_pool_size[pa->pool_info] == high_sz) {
+        max_pool_buf.push_back(buf);
+      }
+    }
+
+    // pick highest
+    const BufferInfoNode* suspect = max_pool_buf[rand() % max_pool_buf.size()];

Review comment:
       What is meant by a "suspect" here? 

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rand_r(&_seedp) % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;
+      total_size = total;
+
+      // reached desired size
+      if (total_size <= desired_bytes_) {
+        break;
+      }
+
+    } else {
+      // rollback
+      swap_buffers(swap_i2, swap_i1);
+      pool_allocations = std::move(rollback_pool_allocations);
+      max_pool_size = find_highest(&pool_allocations);
+    }
+
+    std::vector<const BufferInfoNode*> max_pool_buf;
+
+    for (const auto& it : pool_allocations) {
+      const auto* buf = it.first;
+      const auto pa = it.second;
+      size_t high_sz = pa->byte_offset + buf->size_bytes;
+      if (max_pool_size[pa->pool_info] == high_sz) {
+        max_pool_buf.push_back(buf);
+      }
+    }
+
+    // pick highest
+    const BufferInfoNode* suspect = max_pool_buf[rand() % max_pool_buf.size()];
+    PoolAllocation suspect_pa = pool_allocations[suspect];
+
+    std::unordered_map<int, const BufferInfoNode*, _ptr_hash> first_level_set;

Review comment:
       We should use ObjectPtrHash

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

Review comment:
       Typo

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+  unsigned int _seedp = 0;
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;

Review comment:
       remove

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):

Review comment:
       Im not sure what is being tested here. Would you be able to clarify and add a comment ?

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rnd_func() % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;
+      total_size = total;
+
+      // reached desired size
+      if (total_size <= desired_bytes_) {
+        break;
+      }
+
+    } else {
+      // rollback
+      swap_buffers(swap_i2, swap_i1);
+      pool_allocations = std::move(rollback_pool_allocations);
+      max_pool_size = find_highest(&pool_allocations);
+    }
+
+    std::vector<const BufferInfoNode*> max_pool_buf;
+
+    for (const auto& it : pool_allocations) {

Review comment:
       I think this is because we dont do swaps if its not profitable -- therefore last pool allocation attempted is the best one. Am I right ?
   
   We need to provide a extensive documentation in the top of the file as to how this algorithm works. 
   
   If so, let us remove result_pool_allocations as it does not have a meaning/further use.

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);
+
+    // calculate summary
+    size_t total = 0;
+    for (const auto& el : max_pool_size) {
+      total += el.second;
+    }
+    // accept/reject result heuristic
+    if (!total_size ||
+        (total_size > total ||
+         rnd_func() % 100 < static_cast<int>(300 * (total - total_size) / total / attempts))) {
+      // remember winning combination
+      result_pool_allocations = pool_allocations;
+      total_size = total;
+
+      // reached desired size
+      if (total_size <= desired_bytes_) {
+        break;
+      }
+
+    } else {
+      // rollback
+      swap_buffers(swap_i2, swap_i1);
+      pool_allocations = std::move(rollback_pool_allocations);
+      max_pool_size = find_highest(&pool_allocations);
+    }
+
+    std::vector<const BufferInfoNode*> max_pool_buf;
+
+    for (const auto& it : pool_allocations) {
+      const auto* buf = it.first;
+      const auto pa = it.second;
+      size_t high_sz = pa->byte_offset + buf->size_bytes;
+      if (max_pool_size[pa->pool_info] == high_sz) {
+        max_pool_buf.push_back(buf);
+      }
+    }
+
+    // pick highest
+    const BufferInfoNode* suspect = max_pool_buf[rand() % max_pool_buf.size()];
+    PoolAllocation suspect_pa = pool_allocations[suspect];
+
+    std::unordered_map<int, const BufferInfoNode*, _ptr_hash> first_level_set;
+    std::unordered_map<int, const BufferInfoNode*, _ptr_hash> second_level_set;
+
+    auto suspect_pos = _pos(suspect);

Review comment:
       I feel breaking this suspect-based swap candidate selection to a different function and docs might be better. WDYT ?

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):
+    random.seed(0)
+    mem_range = [BufferInfo(str(i), random.randrange(1, 65535), pools) for i in range(random_len)]
+    for mr in mem_range:
+        pr = random.choice(mem_range)
+        while pr in (*mr.conflicts, mr):
+            pr = random.choice(mem_range)
+
+        mr.set_conflicts([*mr.conflicts, pr])
+        pr.set_conflicts([*pr.conflicts, mr])
+
+    fusmp_algo = tvm.get_global_func("tir.usmp.algo.hill_climb")
+    result_map = fusmp_algo(mem_range, 0)
+    _verify_all_conflicts(result_map)
+
+
+def __test_data_alloc_max():
+    intervals = [
+        (0, 159, 2048),
+        (0, 13, 7904),
+        (4, 35, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+    ]
+    return intervals
+
+
+def __test_data_deep_speech():
+    intervals = [
+        (0, 159, 2048),
+        (0, 151, 2048),
+        (0, 13, 7904),
+        (2, 49, 16),
+        (4, 35, 16),
+        (6, 21, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+        (20, 27, 32768),
+        (26, 31, 32768),
+        (30, 35, 32768),
+        (34, 41, 32768),
+        (40, 45, 32768),
+        (44, 49, 32768),
+        (48, 145, 32768),
+        (54, 59, 2048),
+        (58, 483, 4096),
+        (60, 65, 2048),
+        (64, 461, 4096),
+        (66, 71, 2048),
+        (70, 439, 4096),
+        (72, 77, 2048),
+        (76, 417, 4096),
+        (78, 83, 2048),
+        (82, 395, 4096),
+        (84, 89, 2048),
+        (88, 373, 4096),
+        (90, 95, 2048),
+        (94, 351, 4096),
+        (96, 101, 2048),
+        (100, 329, 4096),
+        (102, 107, 2048),
+        (106, 307, 4096),
+        (108, 113, 2048),
+        (112, 285, 4096),
+        (114, 119, 2048),
+        (118, 263, 4096),
+        (120, 125, 2048),
+        (124, 241, 4096),
+        (126, 131, 2048),
+        (130, 219, 4096),
+        (132, 137, 2048),
+        (136, 197, 4096),
+        (138, 143, 2048),
+        (142, 175, 4096),
+        (144, 149, 2048),
+        (148, 153, 4096),
+        (152, 163, 8192),
+        (154, 171, 2048),
+        (156, 181, 2048),
+        (160, 167, 2048),
+        (162, 165, 2048),
+        (168, 171, 2048),
+        (170, 509, 2048),
+        (174, 185, 8192),
+        (176, 193, 2048),
+        (178, 203, 2048),
+        (182, 189, 2048),
+        (184, 187, 2048),
+        (190, 193, 2048),
+        (192, 511, 2048),
+        (196, 207, 8192),
+        (198, 215, 2048),
+        (200, 225, 2048),
+        (204, 211, 2048),
+        (206, 209, 2048),
+        (212, 215, 2048),
+        (214, 513, 2048),
+        (218, 229, 8192),
+        (220, 237, 2048),
+        (222, 247, 2048),
+        (226, 233, 2048),
+        (228, 231, 2048),
+        (234, 237, 2048),
+        (236, 515, 2048),
+        (240, 251, 8192),
+        (242, 259, 2048),
+        (244, 269, 2048),
+        (248, 255, 2048),
+        (250, 253, 2048),
+        (256, 259, 2048),
+        (258, 517, 2048),
+        (262, 273, 8192),
+        (264, 281, 2048),
+        (266, 291, 2048),
+        (270, 277, 2048),
+        (272, 275, 2048),
+        (278, 281, 2048),
+        (280, 519, 2048),
+        (284, 295, 8192),
+        (286, 303, 2048),
+        (288, 313, 2048),
+        (292, 299, 2048),
+        (294, 297, 2048),
+        (300, 303, 2048),
+        (302, 521, 2048),
+        (306, 317, 8192),
+        (308, 325, 2048),
+        (310, 335, 2048),
+        (314, 321, 2048),
+        (316, 319, 2048),
+        (322, 325, 2048),
+        (324, 523, 2048),
+        (328, 339, 8192),
+        (330, 347, 2048),
+        (332, 357, 2048),
+        (336, 343, 2048),
+        (338, 341, 2048),
+        (344, 347, 2048),
+        (346, 525, 2048),
+        (350, 361, 8192),
+        (352, 369, 2048),
+        (354, 379, 2048),
+        (358, 365, 2048),
+        (360, 363, 2048),
+        (366, 369, 2048),
+        (368, 527, 2048),
+        (372, 383, 8192),
+        (374, 391, 2048),
+        (376, 401, 2048),
+        (380, 387, 2048),
+        (382, 385, 2048),
+        (388, 391, 2048),
+        (390, 529, 2048),
+        (394, 405, 8192),
+        (396, 413, 2048),
+        (398, 423, 2048),
+        (402, 409, 2048),
+        (404, 407, 2048),
+        (410, 413, 2048),
+        (412, 531, 2048),
+        (416, 427, 8192),
+        (418, 435, 2048),
+        (420, 445, 2048),
+        (424, 431, 2048),
+        (426, 429, 2048),
+        (432, 435, 2048),
+        (434, 533, 2048),
+        (438, 449, 8192),
+        (440, 457, 2048),
+        (442, 467, 2048),
+        (446, 453, 2048),
+        (448, 451, 2048),
+        (454, 457, 2048),
+        (456, 535, 2048),
+        (460, 471, 8192),
+        (462, 479, 2048),
+        (464, 489, 2048),
+        (468, 475, 2048),
+        (470, 473, 2048),
+        (476, 479, 2048),
+        (478, 537, 2048),
+        (482, 493, 8192),
+        (484, 501, 2048),
+        (486, 497, 2048),
+        (490, 497, 2048),
+        (492, 495, 2048),
+        (496, 626, 2048),
+        (498, 501, 2048),
+        (500, 626, 2048),
+        (504, 549, 16),
+        (508, 543, 32768),
+        (542, 549, 32768),
+        (548, 555, 32768),
+        (554, 563, 464),
+        (560, 563, 256),
+        (562, 617, 2048),
+        (564, 567, 1856),
+        (566, 573, 1024),
+        (568, 619, 1024),
+        (570, 573, 1024),
+        (572, 577, 1024),
+        (576, 579, 1024),
+        (578, 605, 1024),
+        (580, 593, 1024),
+        (584, 587, 1024),
+        (586, 603, 1024),
+        (594, 597, 1024),
+        (596, 613, 1024),
+        (604, 607, 1024),
+        (606, 617, 1024),
+        (616, 621, 2048),
+        (618, 621, 1024),
+        (620, 626, 464),
+    ]
+    return intervals
+
+
+def __test_data_five():
+    return [
+        (4, 5, 95),
+        (1, 4, 52135),
+        (3, 4, 12136),
+        (3, 5, 62099),
+        (4, 5, 50458),
+    ]
+
+
+def __test_data_simple():
+    return [
+        (0, 23, 131072),  # 0
+        (4, 5, 65568),  # 1
+        (4, 9, 8192),  # 2
+        (8, 30, 15360),  # 3
+        (10, 11, 65568),  # 4
+        (10, 15, 4096),  # 5
+        (16, 17, 65552),  # 6
+        (16, 21, 2048),  # 7
+        (22, 23, 32784),  # 8
+        (22, 27, 1024),  # 9
+    ]
+
+
+def maximumFromIntervals(intervals):
+    # expected list of intervals of (start, end, size)
+    sorted_list = sorted(intervals, key=lambda _: _[0])
+    max_mem = 0
+    for t in range(sorted_list[0][0], sorted_list[-1][1] + 1):
+        max_mem = max(
+            max_mem, sum([size for (start, end, size) in sorted_list if t >= start and t <= end])
+        )
+    return max_mem
+
+
+@pytest.mark.parametrize(
+    "intervals",
+    [__test_data_alloc_max(), __test_data_simple(), __test_data_deep_speech(), __test_data_five()],
+)
+def test_intervals(intervals):
+    result = run_intervals(intervals)
+    assert result["tir.usmp.algo.hill_climb"] == True, f" {result}"
+
+
+def generate_range(sz, max_segment_sz=65535):
+    for i in range(0, sz):
+        start = random.randrange(i, sz)
+        stop = random.randrange(start + 1, start + 2 + ((sz - start) // 2))
+        assert stop - start > 0
+        yield (start, stop, random.randrange(1, max_segment_sz))
+
+
+@pytest.mark.skip()
+def test_10_random_intervals():
+    __test_n_random_intervals(10)
+
+
+@pytest.mark.skip()

Review comment:
       Do we need these anymore ?

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):
+    random.seed(0)
+    mem_range = [BufferInfo(str(i), random.randrange(1, 65535), pools) for i in range(random_len)]
+    for mr in mem_range:
+        pr = random.choice(mem_range)
+        while pr in (*mr.conflicts, mr):
+            pr = random.choice(mem_range)
+
+        mr.set_conflicts([*mr.conflicts, pr])
+        pr.set_conflicts([*pr.conflicts, mr])
+
+    fusmp_algo = tvm.get_global_func("tir.usmp.algo.hill_climb")
+    result_map = fusmp_algo(mem_range, 0)
+    _verify_all_conflicts(result_map)
+
+
+def __test_data_alloc_max():
+    intervals = [
+        (0, 159, 2048),
+        (0, 13, 7904),
+        (4, 35, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+    ]
+    return intervals
+
+
+def __test_data_deep_speech():
+    intervals = [
+        (0, 159, 2048),
+        (0, 151, 2048),
+        (0, 13, 7904),
+        (2, 49, 16),
+        (4, 35, 16),
+        (6, 21, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+        (20, 27, 32768),
+        (26, 31, 32768),
+        (30, 35, 32768),
+        (34, 41, 32768),
+        (40, 45, 32768),
+        (44, 49, 32768),
+        (48, 145, 32768),
+        (54, 59, 2048),
+        (58, 483, 4096),
+        (60, 65, 2048),
+        (64, 461, 4096),
+        (66, 71, 2048),
+        (70, 439, 4096),
+        (72, 77, 2048),
+        (76, 417, 4096),
+        (78, 83, 2048),
+        (82, 395, 4096),
+        (84, 89, 2048),
+        (88, 373, 4096),
+        (90, 95, 2048),
+        (94, 351, 4096),
+        (96, 101, 2048),
+        (100, 329, 4096),
+        (102, 107, 2048),
+        (106, 307, 4096),
+        (108, 113, 2048),
+        (112, 285, 4096),
+        (114, 119, 2048),
+        (118, 263, 4096),
+        (120, 125, 2048),
+        (124, 241, 4096),
+        (126, 131, 2048),
+        (130, 219, 4096),
+        (132, 137, 2048),
+        (136, 197, 4096),
+        (138, 143, 2048),
+        (142, 175, 4096),
+        (144, 149, 2048),
+        (148, 153, 4096),
+        (152, 163, 8192),
+        (154, 171, 2048),
+        (156, 181, 2048),
+        (160, 167, 2048),
+        (162, 165, 2048),
+        (168, 171, 2048),
+        (170, 509, 2048),
+        (174, 185, 8192),
+        (176, 193, 2048),
+        (178, 203, 2048),
+        (182, 189, 2048),
+        (184, 187, 2048),
+        (190, 193, 2048),
+        (192, 511, 2048),
+        (196, 207, 8192),
+        (198, 215, 2048),
+        (200, 225, 2048),
+        (204, 211, 2048),
+        (206, 209, 2048),
+        (212, 215, 2048),
+        (214, 513, 2048),
+        (218, 229, 8192),
+        (220, 237, 2048),
+        (222, 247, 2048),
+        (226, 233, 2048),
+        (228, 231, 2048),
+        (234, 237, 2048),
+        (236, 515, 2048),
+        (240, 251, 8192),
+        (242, 259, 2048),
+        (244, 269, 2048),
+        (248, 255, 2048),
+        (250, 253, 2048),
+        (256, 259, 2048),
+        (258, 517, 2048),
+        (262, 273, 8192),
+        (264, 281, 2048),
+        (266, 291, 2048),
+        (270, 277, 2048),
+        (272, 275, 2048),
+        (278, 281, 2048),
+        (280, 519, 2048),
+        (284, 295, 8192),
+        (286, 303, 2048),
+        (288, 313, 2048),
+        (292, 299, 2048),
+        (294, 297, 2048),
+        (300, 303, 2048),
+        (302, 521, 2048),
+        (306, 317, 8192),
+        (308, 325, 2048),
+        (310, 335, 2048),
+        (314, 321, 2048),
+        (316, 319, 2048),
+        (322, 325, 2048),
+        (324, 523, 2048),
+        (328, 339, 8192),
+        (330, 347, 2048),
+        (332, 357, 2048),
+        (336, 343, 2048),
+        (338, 341, 2048),
+        (344, 347, 2048),
+        (346, 525, 2048),
+        (350, 361, 8192),
+        (352, 369, 2048),
+        (354, 379, 2048),
+        (358, 365, 2048),
+        (360, 363, 2048),
+        (366, 369, 2048),
+        (368, 527, 2048),
+        (372, 383, 8192),
+        (374, 391, 2048),
+        (376, 401, 2048),
+        (380, 387, 2048),
+        (382, 385, 2048),
+        (388, 391, 2048),
+        (390, 529, 2048),
+        (394, 405, 8192),
+        (396, 413, 2048),
+        (398, 423, 2048),
+        (402, 409, 2048),
+        (404, 407, 2048),
+        (410, 413, 2048),
+        (412, 531, 2048),
+        (416, 427, 8192),
+        (418, 435, 2048),
+        (420, 445, 2048),
+        (424, 431, 2048),
+        (426, 429, 2048),
+        (432, 435, 2048),
+        (434, 533, 2048),
+        (438, 449, 8192),
+        (440, 457, 2048),
+        (442, 467, 2048),
+        (446, 453, 2048),
+        (448, 451, 2048),
+        (454, 457, 2048),
+        (456, 535, 2048),
+        (460, 471, 8192),
+        (462, 479, 2048),
+        (464, 489, 2048),
+        (468, 475, 2048),
+        (470, 473, 2048),
+        (476, 479, 2048),
+        (478, 537, 2048),
+        (482, 493, 8192),
+        (484, 501, 2048),
+        (486, 497, 2048),
+        (490, 497, 2048),
+        (492, 495, 2048),
+        (496, 626, 2048),
+        (498, 501, 2048),
+        (500, 626, 2048),
+        (504, 549, 16),
+        (508, 543, 32768),
+        (542, 549, 32768),
+        (548, 555, 32768),
+        (554, 563, 464),
+        (560, 563, 256),
+        (562, 617, 2048),
+        (564, 567, 1856),
+        (566, 573, 1024),
+        (568, 619, 1024),
+        (570, 573, 1024),
+        (572, 577, 1024),
+        (576, 579, 1024),
+        (578, 605, 1024),
+        (580, 593, 1024),
+        (584, 587, 1024),
+        (586, 603, 1024),
+        (594, 597, 1024),
+        (596, 613, 1024),
+        (604, 607, 1024),
+        (606, 617, 1024),
+        (616, 621, 2048),
+        (618, 621, 1024),
+        (620, 626, 464),
+    ]
+    return intervals
+
+
+def __test_data_five():
+    return [
+        (4, 5, 95),
+        (1, 4, 52135),
+        (3, 4, 12136),
+        (3, 5, 62099),
+        (4, 5, 50458),
+    ]
+
+
+def __test_data_simple():
+    return [
+        (0, 23, 131072),  # 0
+        (4, 5, 65568),  # 1
+        (4, 9, 8192),  # 2
+        (8, 30, 15360),  # 3
+        (10, 11, 65568),  # 4
+        (10, 15, 4096),  # 5
+        (16, 17, 65552),  # 6
+        (16, 21, 2048),  # 7
+        (22, 23, 32784),  # 8
+        (22, 27, 1024),  # 9
+    ]
+
+
+def maximumFromIntervals(intervals):
+    # expected list of intervals of (start, end, size)
+    sorted_list = sorted(intervals, key=lambda _: _[0])
+    max_mem = 0
+    for t in range(sorted_list[0][0], sorted_list[-1][1] + 1):
+        max_mem = max(
+            max_mem, sum([size for (start, end, size) in sorted_list if t >= start and t <= end])
+        )
+    return max_mem
+
+
+@pytest.mark.parametrize(
+    "intervals",
+    [__test_data_alloc_max(), __test_data_simple(), __test_data_deep_speech(), __test_data_five()],
+)
+def test_intervals(intervals):

Review comment:
       Would it be possible list test_* functions to the bottom of the file ? It would help the reader understand what are tests and what are helpers. Also, given the number of functions in the file, docstring to each of these functions should be useful

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )

Review comment:
       Remove

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);

Review comment:
       Let us not use "auto" here, the variable name can easily confuse a reader to think it is an integer.

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):
+    random.seed(0)
+    mem_range = [BufferInfo(str(i), random.randrange(1, 65535), pools) for i in range(random_len)]
+    for mr in mem_range:
+        pr = random.choice(mem_range)
+        while pr in (*mr.conflicts, mr):
+            pr = random.choice(mem_range)
+
+        mr.set_conflicts([*mr.conflicts, pr])
+        pr.set_conflicts([*pr.conflicts, mr])
+
+    fusmp_algo = tvm.get_global_func("tir.usmp.algo.hill_climb")
+    result_map = fusmp_algo(mem_range, 0)
+    _verify_all_conflicts(result_map)
+
+
+def __test_data_alloc_max():
+    intervals = [
+        (0, 159, 2048),
+        (0, 13, 7904),
+        (4, 35, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+    ]
+    return intervals
+
+
+def __test_data_deep_speech():
+    intervals = [
+        (0, 159, 2048),
+        (0, 151, 2048),
+        (0, 13, 7904),
+        (2, 49, 16),
+        (4, 35, 16),
+        (6, 21, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+        (20, 27, 32768),
+        (26, 31, 32768),
+        (30, 35, 32768),
+        (34, 41, 32768),
+        (40, 45, 32768),
+        (44, 49, 32768),
+        (48, 145, 32768),
+        (54, 59, 2048),
+        (58, 483, 4096),
+        (60, 65, 2048),
+        (64, 461, 4096),
+        (66, 71, 2048),
+        (70, 439, 4096),
+        (72, 77, 2048),
+        (76, 417, 4096),
+        (78, 83, 2048),
+        (82, 395, 4096),
+        (84, 89, 2048),
+        (88, 373, 4096),
+        (90, 95, 2048),
+        (94, 351, 4096),
+        (96, 101, 2048),
+        (100, 329, 4096),
+        (102, 107, 2048),
+        (106, 307, 4096),
+        (108, 113, 2048),
+        (112, 285, 4096),
+        (114, 119, 2048),
+        (118, 263, 4096),
+        (120, 125, 2048),
+        (124, 241, 4096),
+        (126, 131, 2048),
+        (130, 219, 4096),
+        (132, 137, 2048),
+        (136, 197, 4096),
+        (138, 143, 2048),
+        (142, 175, 4096),
+        (144, 149, 2048),
+        (148, 153, 4096),
+        (152, 163, 8192),
+        (154, 171, 2048),
+        (156, 181, 2048),
+        (160, 167, 2048),
+        (162, 165, 2048),
+        (168, 171, 2048),
+        (170, 509, 2048),
+        (174, 185, 8192),
+        (176, 193, 2048),
+        (178, 203, 2048),
+        (182, 189, 2048),
+        (184, 187, 2048),
+        (190, 193, 2048),
+        (192, 511, 2048),
+        (196, 207, 8192),
+        (198, 215, 2048),
+        (200, 225, 2048),
+        (204, 211, 2048),
+        (206, 209, 2048),
+        (212, 215, 2048),
+        (214, 513, 2048),
+        (218, 229, 8192),
+        (220, 237, 2048),
+        (222, 247, 2048),
+        (226, 233, 2048),
+        (228, 231, 2048),
+        (234, 237, 2048),
+        (236, 515, 2048),
+        (240, 251, 8192),
+        (242, 259, 2048),
+        (244, 269, 2048),
+        (248, 255, 2048),
+        (250, 253, 2048),
+        (256, 259, 2048),
+        (258, 517, 2048),
+        (262, 273, 8192),
+        (264, 281, 2048),
+        (266, 291, 2048),
+        (270, 277, 2048),
+        (272, 275, 2048),
+        (278, 281, 2048),
+        (280, 519, 2048),
+        (284, 295, 8192),
+        (286, 303, 2048),
+        (288, 313, 2048),
+        (292, 299, 2048),
+        (294, 297, 2048),
+        (300, 303, 2048),
+        (302, 521, 2048),
+        (306, 317, 8192),
+        (308, 325, 2048),
+        (310, 335, 2048),
+        (314, 321, 2048),
+        (316, 319, 2048),
+        (322, 325, 2048),
+        (324, 523, 2048),
+        (328, 339, 8192),
+        (330, 347, 2048),
+        (332, 357, 2048),
+        (336, 343, 2048),
+        (338, 341, 2048),
+        (344, 347, 2048),
+        (346, 525, 2048),
+        (350, 361, 8192),
+        (352, 369, 2048),
+        (354, 379, 2048),
+        (358, 365, 2048),
+        (360, 363, 2048),
+        (366, 369, 2048),
+        (368, 527, 2048),
+        (372, 383, 8192),
+        (374, 391, 2048),
+        (376, 401, 2048),
+        (380, 387, 2048),
+        (382, 385, 2048),
+        (388, 391, 2048),
+        (390, 529, 2048),
+        (394, 405, 8192),
+        (396, 413, 2048),
+        (398, 423, 2048),
+        (402, 409, 2048),
+        (404, 407, 2048),
+        (410, 413, 2048),
+        (412, 531, 2048),
+        (416, 427, 8192),
+        (418, 435, 2048),
+        (420, 445, 2048),
+        (424, 431, 2048),
+        (426, 429, 2048),
+        (432, 435, 2048),
+        (434, 533, 2048),
+        (438, 449, 8192),
+        (440, 457, 2048),
+        (442, 467, 2048),
+        (446, 453, 2048),
+        (448, 451, 2048),
+        (454, 457, 2048),
+        (456, 535, 2048),
+        (460, 471, 8192),
+        (462, 479, 2048),
+        (464, 489, 2048),
+        (468, 475, 2048),
+        (470, 473, 2048),
+        (476, 479, 2048),
+        (478, 537, 2048),
+        (482, 493, 8192),
+        (484, 501, 2048),
+        (486, 497, 2048),
+        (490, 497, 2048),
+        (492, 495, 2048),
+        (496, 626, 2048),
+        (498, 501, 2048),
+        (500, 626, 2048),
+        (504, 549, 16),
+        (508, 543, 32768),
+        (542, 549, 32768),
+        (548, 555, 32768),
+        (554, 563, 464),
+        (560, 563, 256),
+        (562, 617, 2048),
+        (564, 567, 1856),
+        (566, 573, 1024),
+        (568, 619, 1024),
+        (570, 573, 1024),
+        (572, 577, 1024),
+        (576, 579, 1024),
+        (578, 605, 1024),
+        (580, 593, 1024),
+        (584, 587, 1024),
+        (586, 603, 1024),
+        (594, 597, 1024),
+        (596, 613, 1024),
+        (604, 607, 1024),
+        (606, 617, 1024),
+        (616, 621, 2048),
+        (618, 621, 1024),
+        (620, 626, 464),
+    ]
+    return intervals
+
+
+def __test_data_five():
+    return [
+        (4, 5, 95),
+        (1, 4, 52135),
+        (3, 4, 12136),
+        (3, 5, 62099),
+        (4, 5, 50458),
+    ]
+
+
+def __test_data_simple():
+    return [
+        (0, 23, 131072),  # 0
+        (4, 5, 65568),  # 1
+        (4, 9, 8192),  # 2
+        (8, 30, 15360),  # 3
+        (10, 11, 65568),  # 4
+        (10, 15, 4096),  # 5
+        (16, 17, 65552),  # 6
+        (16, 21, 2048),  # 7
+        (22, 23, 32784),  # 8
+        (22, 27, 1024),  # 9
+    ]
+
+
+def maximumFromIntervals(intervals):

Review comment:
       Please use python function name styling.

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):
+    random.seed(0)
+    mem_range = [BufferInfo(str(i), random.randrange(1, 65535), pools) for i in range(random_len)]
+    for mr in mem_range:
+        pr = random.choice(mem_range)
+        while pr in (*mr.conflicts, mr):
+            pr = random.choice(mem_range)
+
+        mr.set_conflicts([*mr.conflicts, pr])
+        pr.set_conflicts([*pr.conflicts, mr])
+
+    fusmp_algo = tvm.get_global_func("tir.usmp.algo.hill_climb")
+    result_map = fusmp_algo(mem_range, 0)
+    _verify_all_conflicts(result_map)
+
+
+def __test_data_alloc_max():
+    intervals = [
+        (0, 159, 2048),
+        (0, 13, 7904),
+        (4, 35, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+    ]
+    return intervals
+
+
+def __test_data_deep_speech():
+    intervals = [
+        (0, 159, 2048),
+        (0, 151, 2048),
+        (0, 13, 7904),
+        (2, 49, 16),
+        (4, 35, 16),
+        (6, 21, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+        (20, 27, 32768),
+        (26, 31, 32768),
+        (30, 35, 32768),
+        (34, 41, 32768),
+        (40, 45, 32768),
+        (44, 49, 32768),
+        (48, 145, 32768),
+        (54, 59, 2048),
+        (58, 483, 4096),
+        (60, 65, 2048),
+        (64, 461, 4096),
+        (66, 71, 2048),
+        (70, 439, 4096),
+        (72, 77, 2048),
+        (76, 417, 4096),
+        (78, 83, 2048),
+        (82, 395, 4096),
+        (84, 89, 2048),
+        (88, 373, 4096),
+        (90, 95, 2048),
+        (94, 351, 4096),
+        (96, 101, 2048),
+        (100, 329, 4096),
+        (102, 107, 2048),
+        (106, 307, 4096),
+        (108, 113, 2048),
+        (112, 285, 4096),
+        (114, 119, 2048),
+        (118, 263, 4096),
+        (120, 125, 2048),
+        (124, 241, 4096),
+        (126, 131, 2048),
+        (130, 219, 4096),
+        (132, 137, 2048),
+        (136, 197, 4096),
+        (138, 143, 2048),
+        (142, 175, 4096),
+        (144, 149, 2048),
+        (148, 153, 4096),
+        (152, 163, 8192),
+        (154, 171, 2048),
+        (156, 181, 2048),
+        (160, 167, 2048),
+        (162, 165, 2048),
+        (168, 171, 2048),
+        (170, 509, 2048),
+        (174, 185, 8192),
+        (176, 193, 2048),
+        (178, 203, 2048),
+        (182, 189, 2048),
+        (184, 187, 2048),
+        (190, 193, 2048),
+        (192, 511, 2048),
+        (196, 207, 8192),
+        (198, 215, 2048),
+        (200, 225, 2048),
+        (204, 211, 2048),
+        (206, 209, 2048),
+        (212, 215, 2048),
+        (214, 513, 2048),
+        (218, 229, 8192),
+        (220, 237, 2048),
+        (222, 247, 2048),
+        (226, 233, 2048),
+        (228, 231, 2048),
+        (234, 237, 2048),
+        (236, 515, 2048),
+        (240, 251, 8192),
+        (242, 259, 2048),
+        (244, 269, 2048),
+        (248, 255, 2048),
+        (250, 253, 2048),
+        (256, 259, 2048),
+        (258, 517, 2048),
+        (262, 273, 8192),
+        (264, 281, 2048),
+        (266, 291, 2048),
+        (270, 277, 2048),
+        (272, 275, 2048),
+        (278, 281, 2048),
+        (280, 519, 2048),
+        (284, 295, 8192),
+        (286, 303, 2048),
+        (288, 313, 2048),
+        (292, 299, 2048),
+        (294, 297, 2048),
+        (300, 303, 2048),
+        (302, 521, 2048),
+        (306, 317, 8192),
+        (308, 325, 2048),
+        (310, 335, 2048),
+        (314, 321, 2048),
+        (316, 319, 2048),
+        (322, 325, 2048),
+        (324, 523, 2048),
+        (328, 339, 8192),
+        (330, 347, 2048),
+        (332, 357, 2048),
+        (336, 343, 2048),
+        (338, 341, 2048),
+        (344, 347, 2048),
+        (346, 525, 2048),
+        (350, 361, 8192),
+        (352, 369, 2048),
+        (354, 379, 2048),
+        (358, 365, 2048),
+        (360, 363, 2048),
+        (366, 369, 2048),
+        (368, 527, 2048),
+        (372, 383, 8192),
+        (374, 391, 2048),
+        (376, 401, 2048),
+        (380, 387, 2048),
+        (382, 385, 2048),
+        (388, 391, 2048),
+        (390, 529, 2048),
+        (394, 405, 8192),
+        (396, 413, 2048),
+        (398, 423, 2048),
+        (402, 409, 2048),
+        (404, 407, 2048),
+        (410, 413, 2048),
+        (412, 531, 2048),
+        (416, 427, 8192),
+        (418, 435, 2048),
+        (420, 445, 2048),
+        (424, 431, 2048),
+        (426, 429, 2048),
+        (432, 435, 2048),
+        (434, 533, 2048),
+        (438, 449, 8192),
+        (440, 457, 2048),
+        (442, 467, 2048),
+        (446, 453, 2048),
+        (448, 451, 2048),
+        (454, 457, 2048),
+        (456, 535, 2048),
+        (460, 471, 8192),
+        (462, 479, 2048),
+        (464, 489, 2048),
+        (468, 475, 2048),
+        (470, 473, 2048),
+        (476, 479, 2048),
+        (478, 537, 2048),
+        (482, 493, 8192),
+        (484, 501, 2048),
+        (486, 497, 2048),
+        (490, 497, 2048),
+        (492, 495, 2048),
+        (496, 626, 2048),
+        (498, 501, 2048),
+        (500, 626, 2048),
+        (504, 549, 16),
+        (508, 543, 32768),
+        (542, 549, 32768),
+        (548, 555, 32768),
+        (554, 563, 464),
+        (560, 563, 256),
+        (562, 617, 2048),
+        (564, 567, 1856),
+        (566, 573, 1024),
+        (568, 619, 1024),
+        (570, 573, 1024),
+        (572, 577, 1024),
+        (576, 579, 1024),
+        (578, 605, 1024),
+        (580, 593, 1024),
+        (584, 587, 1024),
+        (586, 603, 1024),
+        (594, 597, 1024),
+        (596, 613, 1024),
+        (604, 607, 1024),
+        (606, 617, 1024),
+        (616, 621, 2048),
+        (618, 621, 1024),
+        (620, 626, 464),
+    ]
+    return intervals
+
+
+def __test_data_five():
+    return [
+        (4, 5, 95),
+        (1, 4, 52135),
+        (3, 4, 12136),
+        (3, 5, 62099),
+        (4, 5, 50458),
+    ]
+
+
+def __test_data_simple():
+    return [
+        (0, 23, 131072),  # 0
+        (4, 5, 65568),  # 1
+        (4, 9, 8192),  # 2
+        (8, 30, 15360),  # 3
+        (10, 11, 65568),  # 4
+        (10, 15, 4096),  # 5
+        (16, 17, 65552),  # 6
+        (16, 21, 2048),  # 7
+        (22, 23, 32784),  # 8
+        (22, 27, 1024),  # 9
+    ]
+
+
+def maximumFromIntervals(intervals):
+    # expected list of intervals of (start, end, size)
+    sorted_list = sorted(intervals, key=lambda _: _[0])
+    max_mem = 0
+    for t in range(sorted_list[0][0], sorted_list[-1][1] + 1):
+        max_mem = max(
+            max_mem, sum([size for (start, end, size) in sorted_list if t >= start and t <= end])
+        )
+    return max_mem
+
+
+@pytest.mark.parametrize(
+    "intervals",
+    [__test_data_alloc_max(), __test_data_simple(), __test_data_deep_speech(), __test_data_five()],
+)
+def test_intervals(intervals):
+    result = run_intervals(intervals)
+    assert result["tir.usmp.algo.hill_climb"] == True, f" {result}"
+
+
+def generate_range(sz, max_segment_sz=65535):
+    for i in range(0, sz):
+        start = random.randrange(i, sz)
+        stop = random.randrange(start + 1, start + 2 + ((sz - start) // 2))
+        assert stop - start > 0
+        yield (start, stop, random.randrange(1, max_segment_sz))
+
+
+@pytest.mark.skip()
+def test_10_random_intervals():
+    __test_n_random_intervals(10)
+
+
+@pytest.mark.skip()
+def test_100_random_intervals():
+    __test_n_random_intervals(100)
+
+
+def __test_n_random_intervals(n=1):
+    result = {}
+    for i in range(n):
+        result["total_runs"] = i + 1
+        r = test_random_intervals(100)
+        for k, v in r.items():
+            if k in result.keys():
+                result[k] += int(v)
+            else:
+                result[k] = int(v)
+
+    print(result)
+
+
+def test_random_intervals(interval_len=16):
+    intervals = list(generate_range(interval_len))
+    return run_intervals(intervals)
+
+
+def run_intervals(intervals):
+    expected_mem = maximumFromIntervals(intervals)
+    pools = [PoolInfo("default", {})]
+    buffers = []
+    # populate
+    for i, (start, stop, size) in enumerate(intervals):
+        buf = BufferInfo(str(i), size, pools)
+        # buf.set_pool_candidates( ["default"] )
+        buffers.append(buf)
+
+    # intersect
+    for i, (i_start, i_stop, _) in enumerate(intervals):
+        conflicts = set()
+        for j, (j_start, j_stop, _) in enumerate(intervals):
+            start = min(i_start, j_start)
+            stop = max(i_stop, j_stop)
+            i_dur = i_stop - i_start + 1
+            j_dur = j_stop - j_start + 1
+
+            if i != j and (stop - start + 1 < i_dur + j_dur):
+                conflicts.add(buffers[j])
+
+        buffers[i].set_conflicts([c for c in sorted(conflicts, key=lambda c: c.name_hint)])
+
+    result = {}
+    for (alg, params) in [
+        ("tir.usmp.algo.hill_climb", (expected_mem,)),
+        ("tir.usmp.algo.greedy_by_size", (expected_mem,)),
+    ]:
+        fusmp_algo = tvm.get_global_func(alg)
+        print("\n", "started", alg)
+        buffer_info_arr = fusmp_algo(buffers, *params)
+        print()
+
+        _verify_all_conflicts(buffer_info_arr)
+        try:
+            _check_max_workspace_size(buffer_info_arr, pools[0], expected_mem)

Review comment:
       Rather than capturing an exception, we could just return true and false here.

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()
+#endif
+
+  std::vector<BufferInfo> buffer_info_vec;
+  for (const auto& buffer_info : buffer_info_arr) {
+    ICHECK(buffer_info->pool_candidates.size())
+        << "Cannot process buffer \"" << buffer_info->name_hint << "\" with no pool candidates";
+    buffer_info_vec.push_back(std::move(buffer_info));
+  }
+
+  sort_vector(&buffer_info_vec);
+
+  // populate positional index map
+  std::unordered_map<const BufferInfoNode*, int, _ptr_hash> _pos_map;
+  for (size_t index = 0; index < buffer_info_vec.size(); ++index) {
+    _pos_map[buffer_info_vec[index].as<BufferInfoNode>()] = index;
+  }
+
+  // size_t first_attempt_size = 0;
+  size_t total_size = 0;
+  int attempts = 0;
+  // int successful_iteration = 0;
+
+  int swap_i1 = -1;
+  int swap_i2 = -1;
+  size_t desired_bytes_ = desired_bytes;
+  constexpr auto _max_attempts = 500;
+  alloc_map_t rollback_pool_allocations;
+  alloc_map_t result_pool_allocations;
+  alloc_map_t pool_allocations;
+
+  auto swap_buffers = [&buffer_info_vec, &_pos_map](int i1, int i2) {
+    if (i1 == i2) return;
+    auto b1 = buffer_info_vec[i1];
+    auto b2 = buffer_info_vec[i2];
+    buffer_info_vec[i1] = b2;
+    buffer_info_vec[i2] = b1;
+
+    _pos_map[b1.as<BufferInfoNode>()] = i2;
+    _pos_map[b2.as<BufferInfoNode>()] = i1;
+  };
+
+  auto _pos = [&_pos_map](const auto* e) {
+    auto it = _pos_map.find(e);
+    if (it != _pos_map.end()) {
+      return it->second;
+    }
+    LOG(FATAL) << "not indexed";
+    return -1;
+  };
+
+  for (; attempts < _max_attempts; ++attempts) {
+    rollback_pool_allocations = std::move(pool_allocations);
+    greedy(&buffer_info_vec, &pool_allocations);
+
+    // estimate result buffers
+    auto max_pool_size = find_highest(&pool_allocations);

Review comment:
       I would suggest using a name "pool_sizes" because max is just the way of finding the pool_size and here I believe all the pool_sizes are present.

##########
File path: tests/python/unittest/test_tir_usmp_algo_hill_climb.py
##########
@@ -0,0 +1,409 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import random
+import tvm
+from tvm.tir.usmp.utils import BufferInfo, PoolInfo
+
+
+def _check_max_workspace_size(buffer_pool_allocations, pool_info, size):
+    max_workspace_size = 0
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        if pool_allocation.pool_info == pool_info:
+            size_candidate = pool_allocation.byte_offset + buffer_info.size_bytes
+            if size_candidate > max_workspace_size:
+                max_workspace_size = size_candidate
+    _diff = max_workspace_size.value - size
+    assert (
+        max_workspace_size.value == size
+    ), "'{}': expected {} got {}, diff {:0.2f}% ({} bytes)".format(
+        pool_info.pool_name, size, max_workspace_size, 100 * _diff / size, _diff
+    )
+
+
+def _verify_conflicts(buffer_info, pool_allocation, buffer_info_map):
+    """helper to check expected liveness conflicts"""
+    for conflict in buffer_info.conflicts:
+        conflict_pool_allocation = buffer_info_map[conflict]
+
+        if conflict_pool_allocation.pool_info == pool_allocation.pool_info:
+            assert conflict_pool_allocation.byte_offset != pool_allocation.byte_offset
+            l2 = (
+                max(
+                    conflict_pool_allocation.byte_offset + conflict.size_bytes,
+                    pool_allocation.byte_offset + buffer_info.size_bytes,
+                )
+                - min(conflict_pool_allocation.byte_offset, pool_allocation.byte_offset)
+            )
+            assert (
+                conflict.size_bytes + buffer_info.size_bytes <= l2
+            ), 'Conflicting: \n"{} @{}"\n"{} @{}"'.format(
+                conflict, conflict_pool_allocation, buffer_info, pool_allocation
+            )
+
+
+def _verify_all_conflicts(buffer_pool_allocations):
+    for buffer_info, pool_allocation in buffer_pool_allocations.items():
+        # print( "Processing ", name )
+        _verify_conflicts(buffer_info, pool_allocation, buffer_pool_allocations)
+
+
+def test_bounded(random_len=150, pools=[PoolInfo("default", {}, 65535), PoolInfo("slow", {})]):
+    random.seed(0)
+    mem_range = [BufferInfo(str(i), random.randrange(1, 65535), pools) for i in range(random_len)]
+    for mr in mem_range:
+        pr = random.choice(mem_range)
+        while pr in (*mr.conflicts, mr):
+            pr = random.choice(mem_range)
+
+        mr.set_conflicts([*mr.conflicts, pr])
+        pr.set_conflicts([*pr.conflicts, mr])
+
+    fusmp_algo = tvm.get_global_func("tir.usmp.algo.hill_climb")
+    result_map = fusmp_algo(mem_range, 0)
+    _verify_all_conflicts(result_map)
+
+
+def __test_data_alloc_max():
+    intervals = [
+        (0, 159, 2048),
+        (0, 13, 7904),
+        (4, 35, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+    ]
+    return intervals
+
+
+def __test_data_deep_speech():
+    intervals = [
+        (0, 159, 2048),
+        (0, 151, 2048),
+        (0, 13, 7904),
+        (2, 49, 16),
+        (4, 35, 16),
+        (6, 21, 16),
+        (12, 17, 32768),
+        (16, 21, 32768),
+        (20, 27, 32768),
+        (26, 31, 32768),
+        (30, 35, 32768),
+        (34, 41, 32768),
+        (40, 45, 32768),
+        (44, 49, 32768),
+        (48, 145, 32768),
+        (54, 59, 2048),
+        (58, 483, 4096),
+        (60, 65, 2048),
+        (64, 461, 4096),
+        (66, 71, 2048),
+        (70, 439, 4096),
+        (72, 77, 2048),
+        (76, 417, 4096),
+        (78, 83, 2048),
+        (82, 395, 4096),
+        (84, 89, 2048),
+        (88, 373, 4096),
+        (90, 95, 2048),
+        (94, 351, 4096),
+        (96, 101, 2048),
+        (100, 329, 4096),
+        (102, 107, 2048),
+        (106, 307, 4096),
+        (108, 113, 2048),
+        (112, 285, 4096),
+        (114, 119, 2048),
+        (118, 263, 4096),
+        (120, 125, 2048),
+        (124, 241, 4096),
+        (126, 131, 2048),
+        (130, 219, 4096),
+        (132, 137, 2048),
+        (136, 197, 4096),
+        (138, 143, 2048),
+        (142, 175, 4096),
+        (144, 149, 2048),
+        (148, 153, 4096),
+        (152, 163, 8192),
+        (154, 171, 2048),
+        (156, 181, 2048),
+        (160, 167, 2048),
+        (162, 165, 2048),
+        (168, 171, 2048),
+        (170, 509, 2048),
+        (174, 185, 8192),
+        (176, 193, 2048),
+        (178, 203, 2048),
+        (182, 189, 2048),
+        (184, 187, 2048),
+        (190, 193, 2048),
+        (192, 511, 2048),
+        (196, 207, 8192),
+        (198, 215, 2048),
+        (200, 225, 2048),
+        (204, 211, 2048),
+        (206, 209, 2048),
+        (212, 215, 2048),
+        (214, 513, 2048),
+        (218, 229, 8192),
+        (220, 237, 2048),
+        (222, 247, 2048),
+        (226, 233, 2048),
+        (228, 231, 2048),
+        (234, 237, 2048),
+        (236, 515, 2048),
+        (240, 251, 8192),
+        (242, 259, 2048),
+        (244, 269, 2048),
+        (248, 255, 2048),
+        (250, 253, 2048),
+        (256, 259, 2048),
+        (258, 517, 2048),
+        (262, 273, 8192),
+        (264, 281, 2048),
+        (266, 291, 2048),
+        (270, 277, 2048),
+        (272, 275, 2048),
+        (278, 281, 2048),
+        (280, 519, 2048),
+        (284, 295, 8192),
+        (286, 303, 2048),
+        (288, 313, 2048),
+        (292, 299, 2048),
+        (294, 297, 2048),
+        (300, 303, 2048),
+        (302, 521, 2048),
+        (306, 317, 8192),
+        (308, 325, 2048),
+        (310, 335, 2048),
+        (314, 321, 2048),
+        (316, 319, 2048),
+        (322, 325, 2048),
+        (324, 523, 2048),
+        (328, 339, 8192),
+        (330, 347, 2048),
+        (332, 357, 2048),
+        (336, 343, 2048),
+        (338, 341, 2048),
+        (344, 347, 2048),
+        (346, 525, 2048),
+        (350, 361, 8192),
+        (352, 369, 2048),
+        (354, 379, 2048),
+        (358, 365, 2048),
+        (360, 363, 2048),
+        (366, 369, 2048),
+        (368, 527, 2048),
+        (372, 383, 8192),
+        (374, 391, 2048),
+        (376, 401, 2048),
+        (380, 387, 2048),
+        (382, 385, 2048),
+        (388, 391, 2048),
+        (390, 529, 2048),
+        (394, 405, 8192),
+        (396, 413, 2048),
+        (398, 423, 2048),
+        (402, 409, 2048),
+        (404, 407, 2048),
+        (410, 413, 2048),
+        (412, 531, 2048),
+        (416, 427, 8192),
+        (418, 435, 2048),
+        (420, 445, 2048),
+        (424, 431, 2048),
+        (426, 429, 2048),
+        (432, 435, 2048),
+        (434, 533, 2048),
+        (438, 449, 8192),
+        (440, 457, 2048),
+        (442, 467, 2048),
+        (446, 453, 2048),
+        (448, 451, 2048),
+        (454, 457, 2048),
+        (456, 535, 2048),
+        (460, 471, 8192),
+        (462, 479, 2048),
+        (464, 489, 2048),
+        (468, 475, 2048),
+        (470, 473, 2048),
+        (476, 479, 2048),
+        (478, 537, 2048),
+        (482, 493, 8192),
+        (484, 501, 2048),
+        (486, 497, 2048),
+        (490, 497, 2048),
+        (492, 495, 2048),
+        (496, 626, 2048),
+        (498, 501, 2048),
+        (500, 626, 2048),
+        (504, 549, 16),
+        (508, 543, 32768),
+        (542, 549, 32768),
+        (548, 555, 32768),
+        (554, 563, 464),
+        (560, 563, 256),
+        (562, 617, 2048),
+        (564, 567, 1856),
+        (566, 573, 1024),
+        (568, 619, 1024),
+        (570, 573, 1024),
+        (572, 577, 1024),
+        (576, 579, 1024),
+        (578, 605, 1024),
+        (580, 593, 1024),
+        (584, 587, 1024),
+        (586, 603, 1024),
+        (594, 597, 1024),
+        (596, 613, 1024),
+        (604, 607, 1024),
+        (606, 617, 1024),
+        (616, 621, 2048),
+        (618, 621, 1024),
+        (620, 626, 464),
+    ]
+    return intervals
+
+
+def __test_data_five():
+    return [
+        (4, 5, 95),
+        (1, 4, 52135),
+        (3, 4, 12136),
+        (3, 5, 62099),
+        (4, 5, 50458),
+    ]
+
+
+def __test_data_simple():
+    return [
+        (0, 23, 131072),  # 0
+        (4, 5, 65568),  # 1
+        (4, 9, 8192),  # 2
+        (8, 30, 15360),  # 3
+        (10, 11, 65568),  # 4
+        (10, 15, 4096),  # 5
+        (16, 17, 65552),  # 6
+        (16, 21, 2048),  # 7
+        (22, 23, 32784),  # 8
+        (22, 27, 1024),  # 9
+    ]
+
+
+def maximumFromIntervals(intervals):
+    # expected list of intervals of (start, end, size)
+    sorted_list = sorted(intervals, key=lambda _: _[0])
+    max_mem = 0
+    for t in range(sorted_list[0][0], sorted_list[-1][1] + 1):
+        max_mem = max(
+            max_mem, sum([size for (start, end, size) in sorted_list if t >= start and t <= end])
+        )
+    return max_mem
+
+
+@pytest.mark.parametrize(
+    "intervals",
+    [__test_data_alloc_max(), __test_data_simple(), __test_data_deep_speech(), __test_data_five()],
+)
+def test_intervals(intervals):
+    result = run_intervals(intervals)
+    assert result["tir.usmp.algo.hill_climb"] == True, f" {result}"
+
+
+def generate_range(sz, max_segment_sz=65535):
+    for i in range(0, sz):
+        start = random.randrange(i, sz)

Review comment:
       We need deterministic testing behaviour. Please include a fixed seed to make this pseudo-random. 

##########
File path: src/tir/usmp/algo/hill_climb.cc
##########
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file tir/analysis/usmp/algo/greedy_by_size.cc
+ * \brief Implement greedy by size memory planning algorithm
+ */
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/function.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/usmp/utils.h>
+
+#include <algorithm>
+#include <numeric>
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+namespace usmp {
+namespace algo {
+/*!
+ * \brief Rounds up the offset to satisfy the alignement requirement
+ */
+static size_t round_up_to_byte_alignment(const size_t& non_aligned_byte_offset,
+                                         const int& byte_alignment) {
+  return ((non_aligned_byte_offset + byte_alignment - 1) / byte_alignment) * byte_alignment;
+}
+
+/*!
+ * \brief A helper function check whether a offset is valid given the constraints
+ */
+static bool IsValidPlacement(const PoolInfo& candidate_pool, const size_t& next_offset,
+                             const size_t& size_bytes) {
+  if (candidate_pool->size_hint_bytes == -1) {
+    // this means pool is not bounded
+    return true;
+  }
+  auto pool_size = static_cast<size_t>(candidate_pool->size_hint_bytes->value);
+  auto max_address = next_offset + size_bytes;
+  if (max_address <= pool_size) {
+    return true;
+  }
+  return false;
+}
+
+/*!
+ * \brief Selects a pool for placement in the given set of ordered pool candidates
+ */
+static PoolInfo SelectPlacementPool(
+    const BufferInfo& buf_info,
+    const std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual>& pool_offsets) {
+  // Here the pool candidates are ordered when it is consumed by the algorithm.
+  // This could be from order the user has specified. However, schedulers are
+  // welcome to change the order for performance reasons.
+  for (const auto& pool_info : buf_info->pool_candidates) {
+    if (pool_offsets.count(pool_info)) {
+      return pool_info;
+    }
+  }
+  CHECK(false) << "TVM USMP Error: the space available in the provided pools exceeded when "
+                  "trying to allocate the buffer : "
+               << buf_info << "\n. Please increase the size_hints for memory pools.";
+  return PoolInfo();
+}
+
+struct _ptr_hash {
+  template <typename T>
+  size_t operator()(const T& a) const {
+    return std::hash<T>()(a);
+  }
+};
+
+using alloc_map_t = std::unordered_map<const BufferInfoNode*, PoolAllocation, _ptr_hash>;
+
+static void sort_vector(std::vector<BufferInfo>* buffer_info_vec) {
+  std::sort(buffer_info_vec->begin(), buffer_info_vec->end(),
+            [](const BufferInfo& a, const BufferInfo& b) {
+              if (a->size_bytes->value == b->size_bytes->value) {
+                if (a->conflicts.size() == b->conflicts.size()) {
+                  auto a_name_hash = std::hash<std::string>{}(a->name_hint->data);
+                  auto b_name_hash = std::hash<std::string>{}(b->name_hint->data);
+                  return a_name_hash > b_name_hash;
+                } else {
+                  return a->conflicts.size() > b->conflicts.size();
+                }
+              }
+              return a->size_bytes->value > b->size_bytes->value;
+            });
+}
+
+/*
+ * Modified version of greedy allocation from greedy_by_size.cc
+ */
+static void greedy(std::vector<BufferInfo>* buffer_info_vec, alloc_map_t* pool_allocations) {
+  for (const auto& buf_info : *buffer_info_vec) {
+    std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> pool_offset_candidates;
+    for (const auto& pool_info : buf_info->pool_candidates) {
+      if (algo::IsValidPlacement(pool_info, 0, buf_info->size_bytes->value)) {
+        pool_offset_candidates[pool_info] = 0;
+      }
+    }
+
+    std::vector<const BufferInfoNode*> buf_conf;
+    for (const auto& conflict_buf_info_obj : buf_info->conflicts) {
+      const BufferInfoNode* conflict_buf_info = conflict_buf_info_obj.as<BufferInfoNode>();
+      if (pool_allocations->end() != pool_allocations->find(conflict_buf_info)) {
+        buf_conf.push_back(conflict_buf_info);
+      }
+    }
+
+    // extra sorting for pool offsets
+    std::sort(buf_conf.begin(), buf_conf.end(), [&pool_allocations](const auto* a, const auto* b) {
+      return pool_allocations->operator[](a)->byte_offset->value <
+             pool_allocations->operator[](b)->byte_offset->value;
+    });
+
+    for (const auto* conflict_buf_info : buf_conf) {
+      size_t next_offset = 0;
+      auto pool_allocation = pool_allocations->operator[](conflict_buf_info);
+      next_offset = pool_allocation->byte_offset + conflict_buf_info->size_bytes;
+      next_offset = round_up_to_byte_alignment(next_offset, conflict_buf_info->alignment->value);
+      if (!pool_offset_candidates.count(pool_allocation->pool_info)) {
+        continue;
+      }
+      if (IsValidPlacement(pool_allocation->pool_info, next_offset, buf_info->size_bytes->value)) {
+        if (next_offset > pool_offset_candidates[pool_allocation->pool_info] &&
+            pool_offset_candidates[pool_allocation->pool_info] +
+                    static_cast<size_t>(buf_info->size_bytes) >
+                static_cast<size_t>(pool_allocation->byte_offset)) {
+          pool_offset_candidates[pool_allocation->pool_info] = next_offset;
+        }
+      } else {
+        pool_offset_candidates.erase(pool_allocation->pool_info);
+      }
+    }
+    auto selected_pool = algo::SelectPlacementPool(buf_info, pool_offset_candidates);
+    pool_allocations->operator[](buf_info.as<BufferInfoNode>()) =
+        PoolAllocation(selected_pool, Integer(pool_offset_candidates[selected_pool]));
+  }
+}
+
+/*
+ * Finds highes allocated memory address for each pool
+ */
+static std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> find_highest(
+    alloc_map_t* pool_allocations) {
+  std::unordered_map<PoolInfo, size_t, ObjectPtrHash, ObjectPtrEqual> max_pool_size;
+  for (const auto it : *pool_allocations) {
+    const BufferInfoNode* buf = it.first;
+    const PoolAllocation& pa = it.second;
+    size_t high_sz = pa->byte_offset + buf->size_bytes;
+    if (max_pool_size[pa->pool_info] <= high_sz) {
+      max_pool_size[pa->pool_info] = high_sz;
+    }
+  }
+  return max_pool_size;
+}
+
+/*
+ * Simulated annealing / Hill climb
+ *
+ * Works by continiously invoking modified 'greedy-by-size' allocation
+ * assessing the result and introduce permutations which hopefully
+ * will led to more 'compact' memory allocation.
+ */
+Map<BufferInfo, PoolAllocation> HillClimb(const Array<BufferInfo>& buffer_info_arr,
+                                          const Integer& desired_bytes) {
+// rand_r does not exist on Windows platform
+#if defined(__linux__) || defined(__ANDROID__)
+  unsigned int _seedp = 0;
+#define rnd_func() rand_r(&_seedp)
+#else
+#define rnd_func() rand()

Review comment:
       We need a deterministic seed for this implementation.




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