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/05/23 17:58:58 UTC

[GitHub] [tvm] junrushao1994 opened a new pull request #8114: [TensorIR][M2a] Verification of cached flags (#390)

junrushao1994 opened a new pull request #8114:
URL: https://github.com/apache/tvm/pull/8114


   This PR is part of the TensorIR upstreaming effort (#7527), stage M2a.
   
   In this PR, we implemented cached flag verification, which will check the correctness of the following flags if the schedule class is set to debug:
   - is_affine_binding
   - region_cover
   - is_stage_pipeline
   
   To enable the detection of the region cover property, we also bring in several integer set analysis along with this PR.


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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/arith/int_set.cc
##########
@@ -694,6 +772,18 @@ IntSet EvalSet(Range r, const std::unordered_map<const VarNode*, IntSet>& dom_ma
   return EvalSet(r, ConvertDomMap(dom_map));
 }
 
+Array<IntSet> EvalSet(const Array<Range>& region, const Map<Var, IntSet>& dom_map) {
+  Analyzer ana;
+  IntervalSetEvaluator m(&ana, dom_map);
+  Array<IntSet> result;
+  result.reserve(region.size());
+  for (const Range& r : region) {
+    PrimExpr sum = r->min + r->extent - 1;

Review comment:
       This will allow constant fold of r->extent -1 and avoid the use of analyzer simplifcation




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

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



[GitHub] [tvm] comaniac commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/tir/schedule/analysis/verify.cc
##########
@@ -142,5 +142,102 @@ class SRefTreeVerifier : public StmtVisitor {
 
 void VerifySRefTree(const ScheduleState& self) { SRefTreeVerifier::Verify(self.get()); }
 
+void VerifyCachedFlags(const ScheduleState& self) {
+  std::vector<StmtSRef> block_info_not_found;
+  std::vector<std::tuple<StmtSRef, bool, bool>> block_info_wrong_affine_binding;
+  std::vector<std::tuple<StmtSRef, bool, bool>> block_info_wrong_region_cover;
+  std::vector<std::tuple<StmtSRef, bool, bool>> block_info_wrong_stage_pipeline;
+
+  ScheduleState new_state(self->mod);
+  for (const auto& kv : new_state->stmt2ref) {
+    const StmtNode* stmt = kv.first;
+    const StmtSRef& new_sref = kv.second;
+    if (stmt->IsInstance<ForNode>() || !self->stmt2ref.count(stmt)) {
+      continue;
+    }
+    const BlockInfo& new_block_info = new_state->block_info.at(new_sref);
+    const StmtSRef& old_sref = self->stmt2ref.at(stmt);
+    if (!self->block_info.count(old_sref)) {
+      block_info_not_found.push_back(new_sref);
+      continue;
+    }
+    const BlockInfo& old_block_info = self->block_info.at(old_sref);
+    if (new_block_info.affine_binding != old_block_info.affine_binding) {
+      block_info_wrong_affine_binding.emplace_back(new_sref,  //
+                                                   new_block_info.affine_binding,
+                                                   old_block_info.affine_binding);
+    }
+    if (new_block_info.region_cover != old_block_info.region_cover) {
+      block_info_wrong_region_cover.emplace_back(new_sref,  //
+                                                 new_block_info.region_cover,
+                                                 old_block_info.region_cover);
+    }
+    if (new_block_info.scope->stage_pipeline != old_block_info.scope->stage_pipeline) {
+      block_info_wrong_stage_pipeline.emplace_back(new_sref,  //
+                                                   new_block_info.scope->stage_pipeline,
+                                                   old_block_info.scope->stage_pipeline);
+    }
+  }
+
+  bool has_not_found = !block_info_not_found.empty();
+  bool has_wrong_affine_binding = !block_info_wrong_affine_binding.empty();
+  bool has_wrong_region_cover = !block_info_wrong_region_cover.empty();
+  bool has_wrong_stage_pipeline = !block_info_wrong_stage_pipeline.empty();
+  if (!(has_not_found || has_wrong_affine_binding || has_wrong_region_cover ||
+        has_wrong_stage_pipeline)) {
+    return;
+  }
+  std::ostringstream os;
+  if (has_not_found) {
+    os << "- BlockInfo not found:";
+    for (const StmtSRef& block_sref : block_info_not_found) {
+      const auto* block = block_sref->StmtAs<BlockNode>();
+      ICHECK(block);
+      os << " " << block->name_hint;
+    }
+    os << std::endl;
+  }
+  if (has_wrong_affine_binding) {
+    os << "- Wrong affine_binding: ";
+    for (const std::tuple<StmtSRef, bool, bool>& record : block_info_wrong_affine_binding) {
+      const StmtSRef& block_sref = std::get<0>(record);
+      bool expected = std::get<1>(record);
+      bool actual = std::get<2>(record);
+      const auto* block = block_sref->StmtAs<BlockNode>();
+      ICHECK(block);
+      os << " (" << block->name_hint << ", expected=" << expected << ", actual=" << actual << ")";
+    }
+    os << std::endl;
+  }
+  if (has_wrong_region_cover) {
+    os << "- Wrong region_cover: ";
+    for (const std::tuple<StmtSRef, bool, bool>& record : block_info_wrong_region_cover) {
+      const StmtSRef& block_sref = std::get<0>(record);
+      bool expected = std::get<1>(record);
+      bool actual = std::get<2>(record);
+      const auto* block = block_sref->StmtAs<BlockNode>();
+      ICHECK(block);
+      os << " (" << block->name_hint << ", expected=" << expected << ", actual=" << actual << ")";
+    }
+    os << std::endl;
+  }
+  if (has_wrong_stage_pipeline) {
+    os << "- Wrong stage_pipeline: ";
+    for (const std::tuple<StmtSRef, bool, bool>& record : block_info_wrong_stage_pipeline) {
+      const StmtSRef& block_sref = std::get<0>(record);
+      bool expected = std::get<1>(record);
+      bool actual = std::get<2>(record);
+      const auto* block = block_sref->StmtAs<BlockNode>();
+      ICHECK(block);
+      os << " (" << block->name_hint << ", expected=" << expected << ", actual=" << actual << ")";
+    }
+    os << std::endl;
+  }
+  LOG(FATAL) << "The schedule failed verification. The IR is:\n"

Review comment:
       ```suggestion
     LOG(FATAL) << "Schedule verification failed. The IR is:\n"
   ```




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/arith/int_set.cc
##########
@@ -635,6 +636,83 @@ IntSet Union(const Array<IntSet>& sets) {
   return IntervalSet(ana.Simplify(x->min_value), ana.Simplify(x->max_value));
 }
 
+Array<IntSet> UnionRegion(const Array<Array<IntSet>>& nd_int_sets) {
+  if (nd_int_sets.empty()) {
+    return {};
+  }
+  int n = nd_int_sets.size();
+  int ndim = nd_int_sets[0].size();
+  Array<IntSet> result;
+  result.reserve(ndim);
+  for (int i = 0; i < ndim; ++i) {
+    Array<IntSet> candidates;
+    candidates.reserve(n);
+    for (int j = 0; j < n; ++j) {
+      candidates.push_back(nd_int_sets[j][i]);
+    }
+    result.push_back(Union(candidates));
+  }
+  return result;
+}
+
+IntSet UnionLowerBound(const Array<IntSet>& sets) {
+  if (sets.size() == 0) return IntSet::Nothing();
+  if (sets.size() == 1) return sets[0];
+  Analyzer analyzer;
+  bool is_first_interval = true;
+  PrimExpr min_inclusive{nullptr};
+  PrimExpr max_exclusive(nullptr);

Review comment:
       To avoid confusion, it might always easier to compute in max_inclusive instead, and the condition instead becomes
   
   ```new_min_inclusive <= max_inclusive + 1```




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/arith/int_set.cc
##########
@@ -635,6 +636,83 @@ IntSet Union(const Array<IntSet>& sets) {
   return IntervalSet(ana.Simplify(x->min_value), ana.Simplify(x->max_value));
 }
 
+Array<IntSet> UnionRegion(const Array<Array<IntSet>>& nd_int_sets) {
+  if (nd_int_sets.empty()) {
+    return {};
+  }
+  int n = nd_int_sets.size();
+  int ndim = nd_int_sets[0].size();
+  Array<IntSet> result;
+  result.reserve(ndim);
+  for (int i = 0; i < ndim; ++i) {
+    Array<IntSet> candidates;
+    candidates.reserve(n);
+    for (int j = 0; j < n; ++j) {
+      candidates.push_back(nd_int_sets[j][i]);
+    }
+    result.push_back(Union(candidates));
+  }
+  return result;
+}
+
+IntSet UnionLowerBound(const Array<IntSet>& sets) {
+  if (sets.size() == 0) return IntSet::Nothing();
+  if (sets.size() == 1) return sets[0];
+  Analyzer analyzer;
+  bool is_first_interval = true;
+  PrimExpr min_inclusive{nullptr};
+  PrimExpr max_exclusive(nullptr);
+  for (const IntSet& int_set : sets) {
+    if (const auto* interval_set = int_set.as<IntervalSetNode>()) {
+      PrimExpr new_min_inclusive = interval_set->min_value;
+      PrimExpr new_max_exclusive = interval_set->max_value;
+      if (!is_pos_inf(new_max_exclusive) && !is_neg_inf(new_max_exclusive)) {
+        new_max_exclusive = new_max_exclusive + 1;
+      }
+      if (is_first_interval) {
+        is_first_interval = false;
+        min_inclusive = std::move(new_min_inclusive);
+        max_exclusive = std::move(new_max_exclusive);
+        continue;
+      }
+      bool bound_1 = is_neg_inf(new_min_inclusive) || is_pos_inf(max_exclusive) ||
+                     analyzer.CanProve(new_min_inclusive <= max_exclusive);
+      bool bound_2 = is_neg_inf(min_inclusive) || is_pos_inf(new_max_exclusive) ||
+                     analyzer.CanProve(min_inclusive <= new_max_exclusive);
+      if (bound_1 && bound_2) {
+        min_inclusive = min(min_inclusive, new_min_inclusive);
+        max_exclusive = max(max_exclusive, new_max_exclusive);
+      }
+    }
+  }
+  if (is_first_interval) {
+    return IntSet::Nothing();
+  }
+  if (!is_neg_inf(max_exclusive) && !is_pos_inf(max_exclusive)) {
+    max_exclusive = max_exclusive - 1;

Review comment:
       consider two separate returns, `return IntSet::Interval(min_inclusive, max_exclusive-1);`




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/tir/schedule/analysis/analysis.cc
##########
@@ -21,6 +21,79 @@
 namespace tvm {
 namespace tir {
 
+/******** Binding ********/
+
+bool IsAffineBinding(const BlockRealize& realize, const Map<Var, Range>& loop_var_ranges,
+                     arith::Analyzer* analyzer) {
+  if (loop_var_ranges.empty()) {
+    return true;
+  }
+  Array<arith::IterSumExpr> results = arith::DetectIterMap(
+      /*indices=*/realize->iter_values,
+      /*input_iters=*/loop_var_ranges,
+      /*predicate=*/realize->predicate,
+      /*require_bijective=*/false,
+      /*analyzer=*/analyzer);
+  if (results.empty()) {
+    return false;
+  }
+  for (const arith::IterSumExpr& sum_expr : results) {
+    const Array<arith::IterSplitExpr>& args = sum_expr->args;
+    if (!args.empty() && !is_one(args[0]->scale)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+Map<Var, Range> LoopDomainOfSRefTreePath(const StmtSRef& low_inclusive,
+                                         const Optional<StmtSRef>& high_exclusive,
+                                         const runtime::StorageScope& extra_relax_scope) {
+  Map<Var, Range> result;
+  const StmtSRefNode* p = low_inclusive.get();
+  const StmtSRefNode* limit = static_cast<const StmtSRefNode*>(high_exclusive.get());
+  for (; p != limit; p = p->parent) {
+    const ForNode* loop = p->StmtAs<ForNode>();
+    if (loop == nullptr) {
+      break;
+    }
+    result.Set(loop->loop_var, Range::FromMinExtent(loop->min, loop->extent));
+  }
+  if (extra_relax_scope.rank != runtime::StorageRank::kGlobal) {
+    for (; p; p = p->parent) {
+      if (const ForNode* loop = p->StmtAs<ForNode>()) {
+        if (loop->kind == ForKind::kThreadBinding) {

Review comment:
       Add a TODO, refactor the relax checking logic into runtime/thread_scope later




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/arith/int_set.cc
##########
@@ -635,6 +636,83 @@ IntSet Union(const Array<IntSet>& sets) {
   return IntervalSet(ana.Simplify(x->min_value), ana.Simplify(x->max_value));
 }
 
+Array<IntSet> UnionRegion(const Array<Array<IntSet>>& nd_int_sets) {
+  if (nd_int_sets.empty()) {
+    return {};
+  }
+  int n = nd_int_sets.size();
+  int ndim = nd_int_sets[0].size();
+  Array<IntSet> result;
+  result.reserve(ndim);
+  for (int i = 0; i < ndim; ++i) {
+    Array<IntSet> candidates;
+    candidates.reserve(n);
+    for (int j = 0; j < n; ++j) {
+      candidates.push_back(nd_int_sets[j][i]);
+    }
+    result.push_back(Union(candidates));
+  }
+  return result;
+}
+
+IntSet UnionLowerBound(const Array<IntSet>& sets) {
+  if (sets.size() == 0) return IntSet::Nothing();
+  if (sets.size() == 1) return sets[0];
+  Analyzer analyzer;
+  bool is_first_interval = true;
+  PrimExpr min_inclusive{nullptr};
+  PrimExpr max_exclusive(nullptr);
+  for (const IntSet& int_set : sets) {
+    if (const auto* interval_set = int_set.as<IntervalSetNode>()) {
+      PrimExpr new_min_inclusive = interval_set->min_value;
+      PrimExpr new_max_exclusive = interval_set->max_value;
+      if (!is_pos_inf(new_max_exclusive) && !is_neg_inf(new_max_exclusive)) {
+        new_max_exclusive = new_max_exclusive + 1;
+      }
+      if (is_first_interval) {
+        is_first_interval = false;
+        min_inclusive = std::move(new_min_inclusive);
+        max_exclusive = std::move(new_max_exclusive);
+        continue;
+      }
+      bool bound_1 = is_neg_inf(new_min_inclusive) || is_pos_inf(max_exclusive) ||
+                     analyzer.CanProve(new_min_inclusive <= max_exclusive);
+      bool bound_2 = is_neg_inf(min_inclusive) || is_pos_inf(new_max_exclusive) ||
+                     analyzer.CanProve(min_inclusive <= new_max_exclusive);
+      if (bound_1 && bound_2) {
+        min_inclusive = min(min_inclusive, new_min_inclusive);

Review comment:
       Consider handle the inf case separately, because min/max may not deal with a case where one of them is inf




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

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



[GitHub] [tvm] comaniac commented on pull request #8114: [TensorIR][M2a] Verification of cached flags

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


   Thanks @junrushao1994  @tqchen 


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

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



[GitHub] [tvm] junrushao1994 commented on pull request #8114: [TensorIR][M2a] Verification of cached flags

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


   CC: @tqchen @jroesch @comaniac @icemelon9 @zhiics


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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/tir/schedule/analysis/analysis.cc
##########
@@ -21,6 +21,79 @@
 namespace tvm {
 namespace tir {
 
+/******** Binding ********/
+
+bool IsAffineBinding(const BlockRealize& realize, const Map<Var, Range>& loop_var_ranges,
+                     arith::Analyzer* analyzer) {
+  if (loop_var_ranges.empty()) {
+    return true;
+  }
+  Array<arith::IterSumExpr> results = arith::DetectIterMap(
+      /*indices=*/realize->iter_values,
+      /*input_iters=*/loop_var_ranges,
+      /*predicate=*/realize->predicate,
+      /*require_bijective=*/false,
+      /*analyzer=*/analyzer);
+  if (results.empty()) {
+    return false;
+  }
+  for (const arith::IterSumExpr& sum_expr : results) {
+    const Array<arith::IterSplitExpr>& args = sum_expr->args;
+    if (!args.empty() && !is_one(args[0]->scale)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+Map<Var, Range> LoopDomainOfSRefTreePath(const StmtSRef& low_inclusive,
+                                         const Optional<StmtSRef>& high_exclusive,
+                                         const runtime::StorageScope& extra_relax_scope) {
+  Map<Var, Range> result;
+  const StmtSRefNode* p = low_inclusive.get();
+  const StmtSRefNode* limit = static_cast<const StmtSRefNode*>(high_exclusive.get());
+  for (; p != limit; p = p->parent) {
+    const ForNode* loop = p->StmtAs<ForNode>();
+    if (loop == nullptr) {
+      break;
+    }
+    result.Set(loop->loop_var, Range::FromMinExtent(loop->min, loop->extent));
+  }
+  if (extra_relax_scope.rank != runtime::StorageRank::kGlobal) {
+    for (; p; p = p->parent) {
+      if (const ForNode* loop = p->StmtAs<ForNode>()) {
+        if (loop->kind == ForKind::kThreadBinding) {

Review comment:
       Add a TODO, refactor the relax checking logic into a single place later.




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

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



[GitHub] [tvm] comaniac merged pull request #8114: [TensorIR][M2a] Verification of cached flags

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


   


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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/tir/schedule/analysis/analysis.cc
##########
@@ -21,6 +21,79 @@
 namespace tvm {
 namespace tir {
 
+/******** Binding ********/
+
+bool IsAffineBinding(const BlockRealize& realize, const Map<Var, Range>& loop_var_ranges,
+                     arith::Analyzer* analyzer) {
+  if (loop_var_ranges.empty()) {
+    return true;
+  }
+  Array<arith::IterSumExpr> results = arith::DetectIterMap(
+      /*indices=*/realize->iter_values,
+      /*input_iters=*/loop_var_ranges,
+      /*predicate=*/realize->predicate,
+      /*require_bijective=*/false,
+      /*analyzer=*/analyzer);
+  if (results.empty()) {
+    return false;
+  }
+  for (const arith::IterSumExpr& sum_expr : results) {
+    const Array<arith::IterSplitExpr>& args = sum_expr->args;
+    if (!args.empty() && !is_one(args[0]->scale)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+Map<Var, Range> LoopDomainOfSRefTreePath(const StmtSRef& low_inclusive,
+                                         const Optional<StmtSRef>& high_exclusive,
+                                         const runtime::StorageScope& extra_relax_scope) {
+  Map<Var, Range> result;
+  const StmtSRefNode* p = low_inclusive.get();
+  const StmtSRefNode* limit = static_cast<const StmtSRefNode*>(high_exclusive.get());
+  for (; p != limit; p = p->parent) {
+    const ForNode* loop = p->StmtAs<ForNode>();
+    if (loop == nullptr) {
+      break;
+    }
+    result.Set(loop->loop_var, Range::FromMinExtent(loop->min, loop->extent));
+  }
+  if (extra_relax_scope.rank != runtime::StorageRank::kGlobal) {
+    for (; p; p = p->parent) {
+      if (const ForNode* loop = p->StmtAs<ForNode>()) {
+        if (loop->kind == ForKind::kThreadBinding) {

Review comment:
       Perhaps I should just implement the logic rather than leaving a TODO for future




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #8114: [TensorIR][M2a] Verification of cached flags

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



##########
File path: src/arith/int_set.cc
##########
@@ -694,6 +772,18 @@ IntSet EvalSet(Range r, const std::unordered_map<const VarNode*, IntSet>& dom_ma
   return EvalSet(r, ConvertDomMap(dom_map));
 }
 
+Array<IntSet> EvalSet(const Array<Range>& region, const Map<Var, IntSet>& dom_map) {
+  Analyzer ana;
+  IntervalSetEvaluator m(&ana, dom_map);
+  Array<IntSet> result;
+  result.reserve(region.size());
+  for (const Range& r : region) {
+    PrimExpr sum = r->min + r->extent - 1;

Review comment:
       r->min + (r->extent -1)




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

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