You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2020/09/18 05:15:03 UTC

[GitHub] [incubator-tvm] merrymercy opened a new pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

merrymercy opened a new pull request #6512:
URL: https://github.com/apache/incubator-tvm/pull/6512


   - Add a tutorial on auto-scheduling a subgraph for GPU
   - Refactor evolutionary search
   - Fix MutateComputeLocation
      - In the old implementation we reuse InitChangeComputeLocation, but this is wrong. Because it always appends new steps and will make the transform history too long. In the new implementation, we only mutate the old steps, so the number of steps keeps the same.
   - Fix micro tvm import issue
   


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -390,135 +383,102 @@ Array<State> SketchPolicyNode::EvolutionarySearch(const Array<State>& init_popul
   Array<State>* pnow = &states_buf1;
   Array<State>* pnext = &states_buf2;
 
-  // The set of explored states to avoid redundancy.
-  std::unordered_set<std::string> explored_set;
-
-  // The heap to maintain the so far best states.
+  // A heap to keep the best states during evolution
   using StateHeapItem = std::pair<State, float>;
   auto cmp = [](const StateHeapItem& left, const StateHeapItem& right) {
     return left.second > right.second;
   };
-  using StateHeap = std::priority_queue<StateHeapItem, std::vector<StateHeapItem>, decltype(cmp)>;
-  StateHeap heap(cmp);
-  auto update_heap = [&heap, &explored_set](const Array<State>& states,
-                                            const std::vector<float>& scores, const int out_size) {
-    float max_score = 0.0;
-    for (size_t i = 0; i < states.size(); ++i) {
-      const State& state = states[i];
+  std::vector<StateHeapItem> heap;
+  std::unordered_set<std::string> in_heap(measured_states_set_);
+  heap.reserve(out_size);
+
+  // auxiliary global variables
+  std::vector<float> pop_scores;
+  std::vector<double> pop_selection_probs;
+  float max_score = 0.0;
+  pop_scores.reserve(population);
+  pop_selection_probs.reserve(population);
+  std::uniform_real_distribution<> dis(0.0, 1.0);
+
+  // mutation rules
+  int mutation_success_ct, mutation_fail_ct;
+  mutation_success_ct = mutation_fail_ct = 0;
+  std::vector<float> rule_weights;
+  std::vector<double> rule_selection_probs;
+  for (const auto& rule : mutation_rules) {
+    rule_weights.push_back(rule->weight);
+  }
+  ComputePrefixSumProb(rule_weights, &rule_selection_probs);
+
+  // Genetic Algorithm
+  for (int k = 0; k < num_iters + 1; ++k) {
+    // Maintain the heap
+    *pnow = search_task->compute_dag.InferBound(*pnow);
+    PruneInvalidState(search_task, pnow);
+    program_cost_model->Predict(search_task, *pnow, &pop_scores);
+
+    for (size_t i = 0; i < pnow->size(); ++i) {
+      const State& state = (*pnow)[i];
       std::string state_str = state.ToStr();
 
-      // Skip redundant states.
-      if (explored_set.count(state_str) > 0) {
-        continue;
-      }
-      explored_set.insert(state_str);
-
-      if (static_cast<int>(heap.size()) < out_size) {
-        // Directly push item if the heap is not full yet.
-        heap.push({state, scores[i]});
-      } else if (scores[i] > heap.top().second) {
-        // Replace the worst state in the heap with the new state.
-        heap.pop();
-        heap.push({state, scores[i]});
+      if (in_heap.count(state_str) == 0) {
+        if (static_cast<int>(heap.size()) < out_size) {
+          heap.emplace_back((*pnow)[i], pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+          in_heap.insert(state_str);
+        } else if (pop_scores[i] > heap.front().second) {
+          std::string old_state_str = heap.front().first.ToStr();
+          in_heap.erase(old_state_str);
+          in_heap.insert(state_str);
+
+          std::pop_heap(heap.begin(), heap.end(), cmp);
+          heap.back() = StateHeapItem(state, pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+        }
+        if (pop_scores[i] > max_score) {
+          max_score = pop_scores[i];
+        }
       }
-      max_score = (scores[i] > max_score) ? scores[i] : max_score;
     }
-    return max_score;
-  };
 
-  // Cost model predicted scores.
-  std::vector<float> scores;
-  scores.reserve(population);
-
-  // The function to generate prefix sum probabilities based on the given scores.
-  auto assign_prob = [](const std::vector<float>& scores, std::vector<double>* prefix_sum_probs) {
-    // Compute selection probabilities.
-    double sum = 0.0;
-    prefix_sum_probs->resize(scores.size());
-    for (size_t i = 0; i < scores.size(); ++i) {
-      sum += std::max(scores[i], 0.0f);
-      (*prefix_sum_probs)[i] = sum;
+    // Print statistical information
+    if (k % 5 == 0 || k == num_iters) {
+      StdCout(verbose) << "GA Iter: " << k << std::fixed << std::setprecision(4)
+                       << "\tMax score: " << max_score << "\tMin score: " << heap.front().second
+                       << "\t#Pop: " << pnow->size() << "\t#M+: " << mutation_success_ct / (k + 1)
+                       << "\t#M-: " << mutation_fail_ct / (k + 1) << std::endl;
     }
-    for (size_t i = 0; i < scores.size(); ++i) {
-      (*prefix_sum_probs)[i] /= sum;
+    if (k == num_iters) {
+      break;
     }
-  };
 
-  // State selection probabilities.
-  std::uniform_real_distribution<> uniform_dist(0.0, 1.0);
-  std::vector<double> state_select_probs;
-  state_select_probs.reserve(population);
+    // Compute selection probability
+    ComputePrefixSumProb(pop_scores, &pop_selection_probs);
 
-  // Mutation rule selection probabilities.
-  std::vector<double> rule_select_probs;
-  rule_select_probs.reserve(mutation_rules.size());
-  std::vector<float> rule_levels;
-  for (const auto& rule : mutation_rules) {
-    rule_levels.push_back(rule->GetLevel(search_task));
-  }
-  assign_prob(rule_levels, &rule_select_probs);
-
-  // Evaluate the init populations.
-  *pnow = search_task->compute_dag.InferBound(*pnow);
-  PruneInvalidState(search_task, pnow);
-  CHECK_GT(pnow->size(), 0) << "All initial populations are invalid";
-  schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-  // Maintain the best states in the heap.
-  float max_score = update_heap(*pnow, scores, out_size);
-
-  // Genetic algorithm.
-  for (auto iter_idx = 1; iter_idx <= num_iters; ++iter_idx) {
-    // Assign the selection probability to each state based on the cost model scores.
-    assign_prob(scores, &state_select_probs);
-
-    // TODO(@comaniac): Perform cross over.
-
-    // Perform mutations.
-    size_t fail_ct = 0;
-    while (pnext->size() < population && fail_ct < population * 2) {
-      // Select a state to be mutated.
-      State tmp_s = (*pnow)[RandomChoose(state_select_probs, &rand_gen)];
-      if (uniform_dist(rand_gen) < mutation_prob) {
-        // Select a rule and mutate the state.
-        const auto& rule = mutation_rules[RandomChoose(rule_select_probs, &rand_gen)];
+    // Do mutation
+    while (pnext->size() < population) {
+      State tmp_s = (*pnow)[RandomChoose(pop_selection_probs, &rand_gen)];
+
+      if (dis(rand_gen) < mutation_prob) {
+        const auto& rule = mutation_rules[RandomChoose(rule_selection_probs, &rand_gen)];
         if (rule->Apply(this, &tmp_s) == PopulationGenerationRule::ResultKind::kValid) {
           pnext->push_back(std::move(tmp_s));
+          mutation_success_ct++;
         } else {
-          fail_ct++;
+          mutation_fail_ct++;
         }
       } else {
-        // Do not mutate this state in this round.
         pnext->push_back(std::move(tmp_s));
       }
     }
 
-    // Evaluate the new populations.
-    *pnext = search_task->compute_dag.InferBound(*pnext);
-    PruneInvalidState(search_task, pnext);
-
-    // Throw away all states generated in this iterations if all new states are invalid.
-    if (pnext->size() > 0) {
-      std::swap(pnext, pnow);
-      schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-      // Maintain the best states in the heap.
-      float iter_max_score = update_heap(*pnow, scores, out_size);
-      max_score = (iter_max_score > max_score) ? iter_max_score : max_score;
-    }
+    std::swap(pnext, pnow);
     pnext->clear();
-
-    if (iter_idx % 5 == 0 || iter_idx == num_iters) {
-      StdCout(verbose) << "GA Iter: " << iter_idx << std::fixed << std::setprecision(4)
-                       << "\tMax Score: " << max_score << "\tPop Size: " << pnow->size()
-                       << std::endl;
-    }
   }
 
-  // Copy best states in the heap to the output.
-  while (!heap.empty()) {
-    auto item = heap.top();
-    heap.pop();

Review comment:
       Ah I missed that the best states are sorted. Thanks for the fix.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -390,135 +383,102 @@ Array<State> SketchPolicyNode::EvolutionarySearch(const Array<State>& init_popul
   Array<State>* pnow = &states_buf1;
   Array<State>* pnext = &states_buf2;
 
-  // The set of explored states to avoid redundancy.
-  std::unordered_set<std::string> explored_set;
-
-  // The heap to maintain the so far best states.
+  // A heap to keep the best states during evolution
   using StateHeapItem = std::pair<State, float>;
   auto cmp = [](const StateHeapItem& left, const StateHeapItem& right) {
     return left.second > right.second;
   };
-  using StateHeap = std::priority_queue<StateHeapItem, std::vector<StateHeapItem>, decltype(cmp)>;
-  StateHeap heap(cmp);
-  auto update_heap = [&heap, &explored_set](const Array<State>& states,
-                                            const std::vector<float>& scores, const int out_size) {
-    float max_score = 0.0;
-    for (size_t i = 0; i < states.size(); ++i) {
-      const State& state = states[i];
+  std::vector<StateHeapItem> heap;
+  std::unordered_set<std::string> in_heap(measured_states_set_);
+  heap.reserve(out_size);
+
+  // auxiliary global variables
+  std::vector<float> pop_scores;
+  std::vector<double> pop_selection_probs;
+  float max_score = 0.0;
+  pop_scores.reserve(population);
+  pop_selection_probs.reserve(population);
+  std::uniform_real_distribution<> dis(0.0, 1.0);
+
+  // mutation rules
+  int mutation_success_ct, mutation_fail_ct;
+  mutation_success_ct = mutation_fail_ct = 0;
+  std::vector<float> rule_weights;
+  std::vector<double> rule_selection_probs;
+  for (const auto& rule : mutation_rules) {
+    rule_weights.push_back(rule->weight);
+  }
+  ComputePrefixSumProb(rule_weights, &rule_selection_probs);
+
+  // Genetic Algorithm
+  for (int k = 0; k < num_iters + 1; ++k) {
+    // Maintain the heap
+    *pnow = search_task->compute_dag.InferBound(*pnow);
+    PruneInvalidState(search_task, pnow);
+    program_cost_model->Predict(search_task, *pnow, &pop_scores);
+
+    for (size_t i = 0; i < pnow->size(); ++i) {
+      const State& state = (*pnow)[i];
       std::string state_str = state.ToStr();
 
-      // Skip redundant states.
-      if (explored_set.count(state_str) > 0) {
-        continue;
-      }
-      explored_set.insert(state_str);
-
-      if (static_cast<int>(heap.size()) < out_size) {
-        // Directly push item if the heap is not full yet.
-        heap.push({state, scores[i]});
-      } else if (scores[i] > heap.top().second) {
-        // Replace the worst state in the heap with the new state.
-        heap.pop();
-        heap.push({state, scores[i]});
+      if (in_heap.count(state_str) == 0) {
+        if (static_cast<int>(heap.size()) < out_size) {
+          heap.emplace_back((*pnow)[i], pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+          in_heap.insert(state_str);
+        } else if (pop_scores[i] > heap.front().second) {
+          std::string old_state_str = heap.front().first.ToStr();
+          in_heap.erase(old_state_str);
+          in_heap.insert(state_str);
+
+          std::pop_heap(heap.begin(), heap.end(), cmp);
+          heap.back() = StateHeapItem(state, pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+        }
+        if (pop_scores[i] > max_score) {
+          max_score = pop_scores[i];
+        }
       }
-      max_score = (scores[i] > max_score) ? scores[i] : max_score;
     }
-    return max_score;
-  };
 
-  // Cost model predicted scores.
-  std::vector<float> scores;
-  scores.reserve(population);
-
-  // The function to generate prefix sum probabilities based on the given scores.
-  auto assign_prob = [](const std::vector<float>& scores, std::vector<double>* prefix_sum_probs) {
-    // Compute selection probabilities.
-    double sum = 0.0;
-    prefix_sum_probs->resize(scores.size());
-    for (size_t i = 0; i < scores.size(); ++i) {
-      sum += std::max(scores[i], 0.0f);
-      (*prefix_sum_probs)[i] = sum;
+    // Print statistical information
+    if (k % 5 == 0 || k == num_iters) {
+      StdCout(verbose) << "GA Iter: " << k << std::fixed << std::setprecision(4)
+                       << "\tMax score: " << max_score << "\tMin score: " << heap.front().second
+                       << "\t#Pop: " << pnow->size() << "\t#M+: " << mutation_success_ct / (k + 1)
+                       << "\t#M-: " << mutation_fail_ct / (k + 1) << std::endl;
     }
-    for (size_t i = 0; i < scores.size(); ++i) {
-      (*prefix_sum_probs)[i] /= sum;
+    if (k == num_iters) {
+      break;
     }
-  };
 
-  // State selection probabilities.
-  std::uniform_real_distribution<> uniform_dist(0.0, 1.0);
-  std::vector<double> state_select_probs;
-  state_select_probs.reserve(population);
+    // Compute selection probability
+    ComputePrefixSumProb(pop_scores, &pop_selection_probs);
 
-  // Mutation rule selection probabilities.
-  std::vector<double> rule_select_probs;
-  rule_select_probs.reserve(mutation_rules.size());
-  std::vector<float> rule_levels;
-  for (const auto& rule : mutation_rules) {
-    rule_levels.push_back(rule->GetLevel(search_task));
-  }
-  assign_prob(rule_levels, &rule_select_probs);
-
-  // Evaluate the init populations.
-  *pnow = search_task->compute_dag.InferBound(*pnow);
-  PruneInvalidState(search_task, pnow);
-  CHECK_GT(pnow->size(), 0) << "All initial populations are invalid";
-  schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-  // Maintain the best states in the heap.
-  float max_score = update_heap(*pnow, scores, out_size);
-
-  // Genetic algorithm.
-  for (auto iter_idx = 1; iter_idx <= num_iters; ++iter_idx) {
-    // Assign the selection probability to each state based on the cost model scores.
-    assign_prob(scores, &state_select_probs);
-
-    // TODO(@comaniac): Perform cross over.
-
-    // Perform mutations.
-    size_t fail_ct = 0;
-    while (pnext->size() < population && fail_ct < population * 2) {
-      // Select a state to be mutated.
-      State tmp_s = (*pnow)[RandomChoose(state_select_probs, &rand_gen)];
-      if (uniform_dist(rand_gen) < mutation_prob) {
-        // Select a rule and mutate the state.
-        const auto& rule = mutation_rules[RandomChoose(rule_select_probs, &rand_gen)];
+    // Do mutation
+    while (pnext->size() < population) {
+      State tmp_s = (*pnow)[RandomChoose(pop_selection_probs, &rand_gen)];
+
+      if (dis(rand_gen) < mutation_prob) {
+        const auto& rule = mutation_rules[RandomChoose(rule_selection_probs, &rand_gen)];
         if (rule->Apply(this, &tmp_s) == PopulationGenerationRule::ResultKind::kValid) {
           pnext->push_back(std::move(tmp_s));
+          mutation_success_ct++;
         } else {
-          fail_ct++;
+          mutation_fail_ct++;
         }
       } else {
-        // Do not mutate this state in this round.
         pnext->push_back(std::move(tmp_s));
       }
     }
 
-    // Evaluate the new populations.
-    *pnext = search_task->compute_dag.InferBound(*pnext);
-    PruneInvalidState(search_task, pnext);
-
-    // Throw away all states generated in this iterations if all new states are invalid.
-    if (pnext->size() > 0) {
-      std::swap(pnext, pnow);
-      schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-      // Maintain the best states in the heap.
-      float iter_max_score = update_heap(*pnow, scores, out_size);
-      max_score = (iter_max_score > max_score) ? iter_max_score : max_score;
-    }
+    std::swap(pnext, pnow);
     pnext->clear();
-
-    if (iter_idx % 5 == 0 || iter_idx == num_iters) {
-      StdCout(verbose) << "GA Iter: " << iter_idx << std::fixed << std::setprecision(4)
-                       << "\tMax Score: " << max_score << "\tPop Size: " << pnow->size()
-                       << std::endl;
-    }
   }
 
-  // Copy best states in the heap to the output.
-  while (!heap.empty()) {
-    auto item = heap.top();
-    heap.pop();

Review comment:
       @comaniac We cannot use `heap.pop` here because we want states with scores from high to low.
   But `heap.pop` will return states with scores from low to high.




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

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



[GitHub] [incubator-tvm] merrymercy commented on pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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


   Comments are addressed. @comaniac @jcf94 @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] [incubator-tvm] merrymercy commented on pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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


   cc @jcf94 @comaniac @junrushao1994 @binarybana 


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

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



[GitHub] [incubator-tvm] comaniac merged pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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


   


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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -390,135 +385,102 @@ Array<State> SketchPolicyNode::EvolutionarySearch(const Array<State>& init_popul
   Array<State>* pnow = &states_buf1;
   Array<State>* pnext = &states_buf2;
 
-  // The set of explored states to avoid redundancy.
-  std::unordered_set<std::string> explored_set;
-
-  // The heap to maintain the so far best states.
+  // A heap to keep the best states during evolution
   using StateHeapItem = std::pair<State, float>;
   auto cmp = [](const StateHeapItem& left, const StateHeapItem& right) {
     return left.second > right.second;
   };
-  using StateHeap = std::priority_queue<StateHeapItem, std::vector<StateHeapItem>, decltype(cmp)>;
-  StateHeap heap(cmp);
-  auto update_heap = [&heap, &explored_set](const Array<State>& states,
-                                            const std::vector<float>& scores, const int out_size) {
-    float max_score = 0.0;
-    for (size_t i = 0; i < states.size(); ++i) {
-      const State& state = states[i];
+  std::vector<StateHeapItem> heap;
+  std::unordered_set<std::string> in_heap(measured_states_set_);
+  heap.reserve(out_size);
+
+  // auxiliary global variables
+  std::vector<float> pop_scores;
+  std::vector<double> pop_selection_probs;
+  float max_score = 0.0;
+  pop_scores.reserve(population);
+  pop_selection_probs.reserve(population);
+  std::uniform_real_distribution<> dis(0.0, 1.0);
+
+  // mutation rules
+  int mutation_success_ct, mutation_fail_ct;
+  mutation_success_ct = mutation_fail_ct = 0;
+  std::vector<float> rule_weights;
+  std::vector<double> rule_selection_probs;
+  for (const auto& rule : mutation_rules) {
+    rule_weights.push_back(rule->weight);
+  }
+  ComputePrefixSumProb(rule_weights, &rule_selection_probs);
+
+  // Genetic Algorithm
+  for (int k = 0; k < num_iters + 1; ++k) {
+    // Maintain the heap
+    *pnow = search_task->compute_dag.InferBound(*pnow);
+    PruneInvalidState(search_task, pnow);

Review comment:
       Yes, I added the check inside `PruneInvalidState`. If all states are invalid, we assume it is an internal error and we can directly kill the program.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -155,19 +169,22 @@ def resume_search(task, log_file):
     sch, args = auto_scheduler.auto_schedule(task, search_policy, tuning_options=tune_option)
 
 
-# resume_search(task, "matmul.json")
+#resume_search(task, "matmul.json")
 
 ######################################################################
 # .. note::
 #   We cannot run the line above because of the conflict between
 #   python's multiprocessing and tvm's thread pool.
-#   After running a tvm generated binary (L112), the python's multiprocessing
-#   library will hang forever.
-#   You have to make sure that you don't run any tvm generated binaries before
-#   calling ansor's search. To run the L156 above, you should comment out L112-114.
+#   After running a tvm generated binary the python's multiprocessing library 
+#   will hang forever. You have to make sure that you don't run any tvm 
+#   generated binaries before calling auot-scheduler's search.
+#   To run the function above, you should comment out all code in 
+#   "Check correctness and evaluate performance" section.
 #
 #   You should be careful about this problem in your applications.
 #   There are other workarounds for this problem.
 #   For example, you can start a new thread/process (with the builtin python library
 #   threading or multiprocessing) and run the tvm binaries in the new thread/process.
 #   This provides an isolation and avoids the conflict in the main thread/process.
+#   You can also use :any:`auto_scheduler.measure.LocalRPCMeasureContext` for auto-scheduler,
+#   as shown in the GPU tutorial (:ref:`auto-scheduler-conv-gpu`).

Review comment:
       Actually, I removed `LocalRPCMeasureContext` to make the major part of the tutorial simpler.
   Because resuming a search is a very advanced usage, most people don't need this part. So we can make it complicated. But all sections above should be as simple as possible.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -390,135 +383,102 @@ Array<State> SketchPolicyNode::EvolutionarySearch(const Array<State>& init_popul
   Array<State>* pnow = &states_buf1;
   Array<State>* pnext = &states_buf2;
 
-  // The set of explored states to avoid redundancy.
-  std::unordered_set<std::string> explored_set;
-
-  // The heap to maintain the so far best states.
+  // A heap to keep the best states during evolution
   using StateHeapItem = std::pair<State, float>;
   auto cmp = [](const StateHeapItem& left, const StateHeapItem& right) {
     return left.second > right.second;
   };
-  using StateHeap = std::priority_queue<StateHeapItem, std::vector<StateHeapItem>, decltype(cmp)>;
-  StateHeap heap(cmp);
-  auto update_heap = [&heap, &explored_set](const Array<State>& states,
-                                            const std::vector<float>& scores, const int out_size) {
-    float max_score = 0.0;
-    for (size_t i = 0; i < states.size(); ++i) {
-      const State& state = states[i];
+  std::vector<StateHeapItem> heap;
+  std::unordered_set<std::string> in_heap(measured_states_set_);
+  heap.reserve(out_size);
+
+  // auxiliary global variables
+  std::vector<float> pop_scores;
+  std::vector<double> pop_selection_probs;
+  float max_score = 0.0;
+  pop_scores.reserve(population);
+  pop_selection_probs.reserve(population);
+  std::uniform_real_distribution<> dis(0.0, 1.0);
+
+  // mutation rules
+  int mutation_success_ct, mutation_fail_ct;
+  mutation_success_ct = mutation_fail_ct = 0;
+  std::vector<float> rule_weights;
+  std::vector<double> rule_selection_probs;
+  for (const auto& rule : mutation_rules) {
+    rule_weights.push_back(rule->weight);
+  }
+  ComputePrefixSumProb(rule_weights, &rule_selection_probs);
+
+  // Genetic Algorithm
+  for (int k = 0; k < num_iters + 1; ++k) {
+    // Maintain the heap
+    *pnow = search_task->compute_dag.InferBound(*pnow);
+    PruneInvalidState(search_task, pnow);
+    program_cost_model->Predict(search_task, *pnow, &pop_scores);
+
+    for (size_t i = 0; i < pnow->size(); ++i) {
+      const State& state = (*pnow)[i];
       std::string state_str = state.ToStr();
 
-      // Skip redundant states.
-      if (explored_set.count(state_str) > 0) {
-        continue;
-      }
-      explored_set.insert(state_str);
-
-      if (static_cast<int>(heap.size()) < out_size) {
-        // Directly push item if the heap is not full yet.
-        heap.push({state, scores[i]});
-      } else if (scores[i] > heap.top().second) {
-        // Replace the worst state in the heap with the new state.
-        heap.pop();
-        heap.push({state, scores[i]});
+      if (in_heap.count(state_str) == 0) {
+        if (static_cast<int>(heap.size()) < out_size) {
+          heap.emplace_back((*pnow)[i], pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+          in_heap.insert(state_str);
+        } else if (pop_scores[i] > heap.front().second) {
+          std::string old_state_str = heap.front().first.ToStr();
+          in_heap.erase(old_state_str);
+          in_heap.insert(state_str);
+
+          std::pop_heap(heap.begin(), heap.end(), cmp);
+          heap.back() = StateHeapItem(state, pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+        }
+        if (pop_scores[i] > max_score) {
+          max_score = pop_scores[i];
+        }
       }
-      max_score = (scores[i] > max_score) ? scores[i] : max_score;
     }
-    return max_score;
-  };
 
-  // Cost model predicted scores.
-  std::vector<float> scores;
-  scores.reserve(population);
-
-  // The function to generate prefix sum probabilities based on the given scores.
-  auto assign_prob = [](const std::vector<float>& scores, std::vector<double>* prefix_sum_probs) {
-    // Compute selection probabilities.
-    double sum = 0.0;
-    prefix_sum_probs->resize(scores.size());
-    for (size_t i = 0; i < scores.size(); ++i) {
-      sum += std::max(scores[i], 0.0f);
-      (*prefix_sum_probs)[i] = sum;
+    // Print statistical information
+    if (k % 5 == 0 || k == num_iters) {
+      StdCout(verbose) << "GA Iter: " << k << std::fixed << std::setprecision(4)
+                       << "\tMax score: " << max_score << "\tMin score: " << heap.front().second
+                       << "\t#Pop: " << pnow->size() << "\t#M+: " << mutation_success_ct / (k + 1)
+                       << "\t#M-: " << mutation_fail_ct / (k + 1) << std::endl;
     }
-    for (size_t i = 0; i < scores.size(); ++i) {
-      (*prefix_sum_probs)[i] /= sum;
+    if (k == num_iters) {
+      break;
     }
-  };
 
-  // State selection probabilities.
-  std::uniform_real_distribution<> uniform_dist(0.0, 1.0);
-  std::vector<double> state_select_probs;
-  state_select_probs.reserve(population);
+    // Compute selection probability
+    ComputePrefixSumProb(pop_scores, &pop_selection_probs);
 
-  // Mutation rule selection probabilities.
-  std::vector<double> rule_select_probs;
-  rule_select_probs.reserve(mutation_rules.size());
-  std::vector<float> rule_levels;
-  for (const auto& rule : mutation_rules) {
-    rule_levels.push_back(rule->GetLevel(search_task));
-  }
-  assign_prob(rule_levels, &rule_select_probs);
-
-  // Evaluate the init populations.
-  *pnow = search_task->compute_dag.InferBound(*pnow);
-  PruneInvalidState(search_task, pnow);
-  CHECK_GT(pnow->size(), 0) << "All initial populations are invalid";
-  schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-  // Maintain the best states in the heap.
-  float max_score = update_heap(*pnow, scores, out_size);
-
-  // Genetic algorithm.
-  for (auto iter_idx = 1; iter_idx <= num_iters; ++iter_idx) {
-    // Assign the selection probability to each state based on the cost model scores.
-    assign_prob(scores, &state_select_probs);
-
-    // TODO(@comaniac): Perform cross over.
-
-    // Perform mutations.
-    size_t fail_ct = 0;
-    while (pnext->size() < population && fail_ct < population * 2) {
-      // Select a state to be mutated.
-      State tmp_s = (*pnow)[RandomChoose(state_select_probs, &rand_gen)];
-      if (uniform_dist(rand_gen) < mutation_prob) {
-        // Select a rule and mutate the state.
-        const auto& rule = mutation_rules[RandomChoose(rule_select_probs, &rand_gen)];
+    // Do mutation
+    while (pnext->size() < population) {
+      State tmp_s = (*pnow)[RandomChoose(pop_selection_probs, &rand_gen)];
+
+      if (dis(rand_gen) < mutation_prob) {
+        const auto& rule = mutation_rules[RandomChoose(rule_selection_probs, &rand_gen)];
         if (rule->Apply(this, &tmp_s) == PopulationGenerationRule::ResultKind::kValid) {
           pnext->push_back(std::move(tmp_s));
+          mutation_success_ct++;
         } else {
-          fail_ct++;
+          mutation_fail_ct++;
         }
       } else {
-        // Do not mutate this state in this round.
         pnext->push_back(std::move(tmp_s));
       }
     }
 
-    // Evaluate the new populations.
-    *pnext = search_task->compute_dag.InferBound(*pnext);
-    PruneInvalidState(search_task, pnext);
-
-    // Throw away all states generated in this iterations if all new states are invalid.
-    if (pnext->size() > 0) {
-      std::swap(pnext, pnow);
-      schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-      // Maintain the best states in the heap.
-      float iter_max_score = update_heap(*pnow, scores, out_size);
-      max_score = (iter_max_score > max_score) ? iter_max_score : max_score;
-    }
+    std::swap(pnext, pnow);
     pnext->clear();
-
-    if (iter_idx % 5 == 0 || iter_idx == num_iters) {
-      StdCout(verbose) << "GA Iter: " << iter_idx << std::fixed << std::setprecision(4)
-                       << "\tMax Score: " << max_score << "\tPop Size: " << pnow->size()
-                       << std::endl;
-    }
   }
 
-  // Copy best states in the heap to the output.
-  while (!heap.empty()) {
-    auto item = heap.top();
-    heap.pop();

Review comment:
       @comaniac  We cannot use `heap.pop` here because we want the states with scores from high to low.
   But `heap.pop` will give us the states with scores from low to high.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: python/tvm/auto_scheduler/measure.py
##########
@@ -25,8 +25,8 @@
 A builder builds the executable binary files and a runner runs the binary files to
 get the measurement results. The flow of data structures is
 
-                `ProgramBuilder`                 `ProgramRunner`
-`MeasureInput` -----------------> `BuildResult` ----------------> `MeasureResult`
+  .                `ProgramBuilder`                 `ProgramRunner`

Review comment:
       To fix an auto doc generation warning.




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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


   Thanks @merrymercy @jcf94 


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -161,13 +178,16 @@ def resume_search(task, log_file):
 # .. note::
 #   We cannot run the line above because of the conflict between
 #   python's multiprocessing and tvm's thread pool.
-#   After running a tvm generated binary (L112), the python's multiprocessing
-#   library will hang forever.
-#   You have to make sure that you don't run any tvm generated binaries before
-#   calling ansor's search. To run the L156 above, you should comment out L112-114.
+#   After running a tvm generated binary the python's multiprocessing library
+#   will hang forever. You have to make sure that you don't run any tvm
+#   generated binaries before calling auot-scheduler's search.
+#   To run the function above, you should comment out all code in
+#   "Check correctness and evaluate performance" section.
 #
 #   You should be careful about this problem in your applications.
 #   There are other workarounds for this problem.
 #   For example, you can start a new thread/process (with the builtin python library
 #   threading or multiprocessing) and run the tvm binaries in the new thread/process.
 #   This provides an isolation and avoids the conflict in the main thread/process.
+#   You can also use :any:`auto_scheduler.measure.LocalRPCMeasureContext` for auto-scheduler,
+#   as shown in the GPU tutorial (:ref:`auto-scheduler-conv-gpu`).

Review comment:
       I missed the comment from @jcf94 when reviewing this part. I agree that this may be confusing, but I prefer to keep the resume part; otherwise users will still encounter this issue when they are trying to resume the search. For example, one may write a script to do the following:
   
   ```python
   sch, args = do_search(task, log_file)
   perf = evaluate_result(sch, args)
   while perf < goal:
     sch, args = resume_search(task, log_file)
     perf = evaluate_result(sch, args)
   ```




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

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



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: python/tvm/auto_scheduler/measure.py
##########
@@ -25,8 +25,8 @@
 A builder builds the executable binary files and a runner runs the binary files to
 get the measurement results. The flow of data structures is
 
-                `ProgramBuilder`                 `ProgramRunner`
-`MeasureInput` -----------------> `BuildResult` ----------------> `MeasureResult`
+  .                `ProgramBuilder`                 `ProgramRunner`

Review comment:
       The "." is for?

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -155,19 +169,22 @@ def resume_search(task, log_file):
     sch, args = auto_scheduler.auto_schedule(task, search_policy, tuning_options=tune_option)
 
 
-# resume_search(task, "matmul.json")
+#resume_search(task, "matmul.json")
 
 ######################################################################
 # .. note::
 #   We cannot run the line above because of the conflict between
 #   python's multiprocessing and tvm's thread pool.
-#   After running a tvm generated binary (L112), the python's multiprocessing
-#   library will hang forever.
-#   You have to make sure that you don't run any tvm generated binaries before
-#   calling ansor's search. To run the L156 above, you should comment out L112-114.
+#   After running a tvm generated binary the python's multiprocessing library 
+#   will hang forever. You have to make sure that you don't run any tvm 
+#   generated binaries before calling auot-scheduler's search.
+#   To run the function above, you should comment out all code in 
+#   "Check correctness and evaluate performance" section.
 #
 #   You should be careful about this problem in your applications.
 #   There are other workarounds for this problem.
 #   For example, you can start a new thread/process (with the builtin python library
 #   threading or multiprocessing) and run the tvm binaries in the new thread/process.
 #   This provides an isolation and avoids the conflict in the main thread/process.
+#   You can also use :any:`auto_scheduler.measure.LocalRPCMeasureContext` for auto-scheduler,
+#   as shown in the GPU tutorial (:ref:`auto-scheduler-conv-gpu`).

Review comment:
       This may still be confusing. I'm thinking it may be better to just delete the resume part of the CPU tutorials to keep it as simple as possible.
   
   We can add a comment saying that `LocalRPCMeasureContext` is recommended to be used in all hardware targets in the GPU tutorial.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -390,135 +383,102 @@ Array<State> SketchPolicyNode::EvolutionarySearch(const Array<State>& init_popul
   Array<State>* pnow = &states_buf1;
   Array<State>* pnext = &states_buf2;
 
-  // The set of explored states to avoid redundancy.
-  std::unordered_set<std::string> explored_set;
-
-  // The heap to maintain the so far best states.
+  // A heap to keep the best states during evolution
   using StateHeapItem = std::pair<State, float>;
   auto cmp = [](const StateHeapItem& left, const StateHeapItem& right) {
     return left.second > right.second;
   };
-  using StateHeap = std::priority_queue<StateHeapItem, std::vector<StateHeapItem>, decltype(cmp)>;
-  StateHeap heap(cmp);
-  auto update_heap = [&heap, &explored_set](const Array<State>& states,
-                                            const std::vector<float>& scores, const int out_size) {
-    float max_score = 0.0;
-    for (size_t i = 0; i < states.size(); ++i) {
-      const State& state = states[i];
+  std::vector<StateHeapItem> heap;
+  std::unordered_set<std::string> in_heap(measured_states_set_);
+  heap.reserve(out_size);
+
+  // auxiliary global variables
+  std::vector<float> pop_scores;
+  std::vector<double> pop_selection_probs;
+  float max_score = 0.0;
+  pop_scores.reserve(population);
+  pop_selection_probs.reserve(population);
+  std::uniform_real_distribution<> dis(0.0, 1.0);
+
+  // mutation rules
+  int mutation_success_ct, mutation_fail_ct;
+  mutation_success_ct = mutation_fail_ct = 0;
+  std::vector<float> rule_weights;
+  std::vector<double> rule_selection_probs;
+  for (const auto& rule : mutation_rules) {
+    rule_weights.push_back(rule->weight);
+  }
+  ComputePrefixSumProb(rule_weights, &rule_selection_probs);
+
+  // Genetic Algorithm
+  for (int k = 0; k < num_iters + 1; ++k) {
+    // Maintain the heap
+    *pnow = search_task->compute_dag.InferBound(*pnow);
+    PruneInvalidState(search_task, pnow);
+    program_cost_model->Predict(search_task, *pnow, &pop_scores);
+
+    for (size_t i = 0; i < pnow->size(); ++i) {
+      const State& state = (*pnow)[i];
       std::string state_str = state.ToStr();
 
-      // Skip redundant states.
-      if (explored_set.count(state_str) > 0) {
-        continue;
-      }
-      explored_set.insert(state_str);
-
-      if (static_cast<int>(heap.size()) < out_size) {
-        // Directly push item if the heap is not full yet.
-        heap.push({state, scores[i]});
-      } else if (scores[i] > heap.top().second) {
-        // Replace the worst state in the heap with the new state.
-        heap.pop();
-        heap.push({state, scores[i]});
+      if (in_heap.count(state_str) == 0) {
+        if (static_cast<int>(heap.size()) < out_size) {
+          heap.emplace_back((*pnow)[i], pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+          in_heap.insert(state_str);
+        } else if (pop_scores[i] > heap.front().second) {
+          std::string old_state_str = heap.front().first.ToStr();
+          in_heap.erase(old_state_str);
+          in_heap.insert(state_str);
+
+          std::pop_heap(heap.begin(), heap.end(), cmp);
+          heap.back() = StateHeapItem(state, pop_scores[i]);
+          std::push_heap(heap.begin(), heap.end(), cmp);
+        }
+        if (pop_scores[i] > max_score) {
+          max_score = pop_scores[i];
+        }
       }
-      max_score = (scores[i] > max_score) ? scores[i] : max_score;
     }
-    return max_score;
-  };
 
-  // Cost model predicted scores.
-  std::vector<float> scores;
-  scores.reserve(population);
-
-  // The function to generate prefix sum probabilities based on the given scores.
-  auto assign_prob = [](const std::vector<float>& scores, std::vector<double>* prefix_sum_probs) {
-    // Compute selection probabilities.
-    double sum = 0.0;
-    prefix_sum_probs->resize(scores.size());
-    for (size_t i = 0; i < scores.size(); ++i) {
-      sum += std::max(scores[i], 0.0f);
-      (*prefix_sum_probs)[i] = sum;
+    // Print statistical information
+    if (k % 5 == 0 || k == num_iters) {
+      StdCout(verbose) << "GA Iter: " << k << std::fixed << std::setprecision(4)
+                       << "\tMax score: " << max_score << "\tMin score: " << heap.front().second
+                       << "\t#Pop: " << pnow->size() << "\t#M+: " << mutation_success_ct / (k + 1)
+                       << "\t#M-: " << mutation_fail_ct / (k + 1) << std::endl;
     }
-    for (size_t i = 0; i < scores.size(); ++i) {
-      (*prefix_sum_probs)[i] /= sum;
+    if (k == num_iters) {
+      break;
     }
-  };
 
-  // State selection probabilities.
-  std::uniform_real_distribution<> uniform_dist(0.0, 1.0);
-  std::vector<double> state_select_probs;
-  state_select_probs.reserve(population);
+    // Compute selection probability
+    ComputePrefixSumProb(pop_scores, &pop_selection_probs);
 
-  // Mutation rule selection probabilities.
-  std::vector<double> rule_select_probs;
-  rule_select_probs.reserve(mutation_rules.size());
-  std::vector<float> rule_levels;
-  for (const auto& rule : mutation_rules) {
-    rule_levels.push_back(rule->GetLevel(search_task));
-  }
-  assign_prob(rule_levels, &rule_select_probs);
-
-  // Evaluate the init populations.
-  *pnow = search_task->compute_dag.InferBound(*pnow);
-  PruneInvalidState(search_task, pnow);
-  CHECK_GT(pnow->size(), 0) << "All initial populations are invalid";
-  schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-  // Maintain the best states in the heap.
-  float max_score = update_heap(*pnow, scores, out_size);
-
-  // Genetic algorithm.
-  for (auto iter_idx = 1; iter_idx <= num_iters; ++iter_idx) {
-    // Assign the selection probability to each state based on the cost model scores.
-    assign_prob(scores, &state_select_probs);
-
-    // TODO(@comaniac): Perform cross over.
-
-    // Perform mutations.
-    size_t fail_ct = 0;
-    while (pnext->size() < population && fail_ct < population * 2) {
-      // Select a state to be mutated.
-      State tmp_s = (*pnow)[RandomChoose(state_select_probs, &rand_gen)];
-      if (uniform_dist(rand_gen) < mutation_prob) {
-        // Select a rule and mutate the state.
-        const auto& rule = mutation_rules[RandomChoose(rule_select_probs, &rand_gen)];
+    // Do mutation
+    while (pnext->size() < population) {
+      State tmp_s = (*pnow)[RandomChoose(pop_selection_probs, &rand_gen)];
+
+      if (dis(rand_gen) < mutation_prob) {
+        const auto& rule = mutation_rules[RandomChoose(rule_selection_probs, &rand_gen)];
         if (rule->Apply(this, &tmp_s) == PopulationGenerationRule::ResultKind::kValid) {
           pnext->push_back(std::move(tmp_s));
+          mutation_success_ct++;
         } else {
-          fail_ct++;
+          mutation_fail_ct++;
         }
       } else {
-        // Do not mutate this state in this round.
         pnext->push_back(std::move(tmp_s));
       }
     }
 
-    // Evaluate the new populations.
-    *pnext = search_task->compute_dag.InferBound(*pnext);
-    PruneInvalidState(search_task, pnext);
-
-    // Throw away all states generated in this iterations if all new states are invalid.
-    if (pnext->size() > 0) {
-      std::swap(pnext, pnow);
-      schedule_cost_model->Predict(search_task, *pnow, &scores);
-
-      // Maintain the best states in the heap.
-      float iter_max_score = update_heap(*pnow, scores, out_size);
-      max_score = (iter_max_score > max_score) ? iter_max_score : max_score;
-    }
+    std::swap(pnext, pnow);
     pnext->clear();
-
-    if (iter_idx % 5 == 0 || iter_idx == num_iters) {
-      StdCout(verbose) << "GA Iter: " << iter_idx << std::fixed << std::setprecision(4)
-                       << "\tMax Score: " << max_score << "\tPop Size: " << pnow->size()
-                       << std::endl;
-    }
   }
 
-  // Copy best states in the heap to the output.
-  while (!heap.empty()) {
-    auto item = heap.top();
-    heap.pop();

Review comment:
       @comaniac  We cannot use heap.pop here because we want the states with scores from high to low.
   But `heap.pop` will give us the states with scores from low to high.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -155,19 +169,22 @@ def resume_search(task, log_file):
     sch, args = auto_scheduler.auto_schedule(task, search_policy, tuning_options=tune_option)
 
 
-# resume_search(task, "matmul.json")
+#resume_search(task, "matmul.json")
 
 ######################################################################
 # .. note::
 #   We cannot run the line above because of the conflict between
 #   python's multiprocessing and tvm's thread pool.
-#   After running a tvm generated binary (L112), the python's multiprocessing
-#   library will hang forever.
-#   You have to make sure that you don't run any tvm generated binaries before
-#   calling ansor's search. To run the L156 above, you should comment out L112-114.
+#   After running a tvm generated binary the python's multiprocessing library 
+#   will hang forever. You have to make sure that you don't run any tvm 
+#   generated binaries before calling auot-scheduler's search.
+#   To run the function above, you should comment out all code in 
+#   "Check correctness and evaluate performance" section.
 #
 #   You should be careful about this problem in your applications.
 #   There are other workarounds for this problem.
 #   For example, you can start a new thread/process (with the builtin python library
 #   threading or multiprocessing) and run the tvm binaries in the new thread/process.
 #   This provides an isolation and avoids the conflict in the main thread/process.
+#   You can also use :any:`auto_scheduler.measure.LocalRPCMeasureContext` for auto-scheduler,
+#   as shown in the GPU tutorial (:ref:`auto-scheduler-conv-gpu`).

Review comment:
       People are likely to get into this problem in their applications, so it is worth to let them aware of this problem in the tutorial.




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6512: [ANSOR] Auto-scheduler tutorial for GPU and necessary refactor/fix

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



##########
File path: tutorials/auto_scheduler/tune_conv2d_layer_cuda.py
##########
@@ -0,0 +1,190 @@
+# 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.
+"""
+.. _auto-scheduler-conv-gpu:
+
+Auto-scheduling a convolution layer for GPU
+=============================================

Review comment:
       ```suggestion
   ===========================================
   ```

##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -390,135 +385,102 @@ Array<State> SketchPolicyNode::EvolutionarySearch(const Array<State>& init_popul
   Array<State>* pnow = &states_buf1;
   Array<State>* pnext = &states_buf2;
 
-  // The set of explored states to avoid redundancy.
-  std::unordered_set<std::string> explored_set;
-
-  // The heap to maintain the so far best states.
+  // A heap to keep the best states during evolution
   using StateHeapItem = std::pair<State, float>;
   auto cmp = [](const StateHeapItem& left, const StateHeapItem& right) {
     return left.second > right.second;
   };
-  using StateHeap = std::priority_queue<StateHeapItem, std::vector<StateHeapItem>, decltype(cmp)>;
-  StateHeap heap(cmp);
-  auto update_heap = [&heap, &explored_set](const Array<State>& states,
-                                            const std::vector<float>& scores, const int out_size) {
-    float max_score = 0.0;
-    for (size_t i = 0; i < states.size(); ++i) {
-      const State& state = states[i];
+  std::vector<StateHeapItem> heap;
+  std::unordered_set<std::string> in_heap(measured_states_set_);
+  heap.reserve(out_size);
+
+  // auxiliary global variables
+  std::vector<float> pop_scores;
+  std::vector<double> pop_selection_probs;
+  float max_score = 0.0;
+  pop_scores.reserve(population);
+  pop_selection_probs.reserve(population);
+  std::uniform_real_distribution<> dis(0.0, 1.0);
+
+  // mutation rules
+  int mutation_success_ct, mutation_fail_ct;
+  mutation_success_ct = mutation_fail_ct = 0;
+  std::vector<float> rule_weights;
+  std::vector<double> rule_selection_probs;
+  for (const auto& rule : mutation_rules) {
+    rule_weights.push_back(rule->weight);
+  }
+  ComputePrefixSumProb(rule_weights, &rule_selection_probs);
+
+  // Genetic Algorithm
+  for (int k = 0; k < num_iters + 1; ++k) {
+    // Maintain the heap
+    *pnow = search_task->compute_dag.InferBound(*pnow);
+    PruneInvalidState(search_task, pnow);

Review comment:
       I moved this part to the end of each GA iteration in order to deal with the case that all generated states are pruned by this call. What happen if all states are invalid and `pnow` becomes empty? Should we simply error out and stop the tuning, as it is likely problematic?

##########
File path: tutorials/auto_scheduler/tune_conv2d_layer_cuda.py
##########
@@ -0,0 +1,190 @@
+# 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.
+"""
+.. _auto-scheduler-conv-gpu:
+
+Auto-scheduling a convolution layer for GPU
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+
+Different from the existing :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use a convolution layer as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler, topi
+from tvm.topi.testing import conv2d_nchw_python
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, let us define the computation of a convolution layer.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def conv2d_layer(N, H, W, CO, CI, KH, KW, stride, padding):
+    data = te.placeholder((N, CI, H, W), name="data")
+    kernel = te.placeholder((CO, CI, KH, KW), name="kernel")
+    bias = te.placeholder((1, CO, 1, 1), name="bias")
+    conv = topi.nn.conv2d_nchw(data, kernel, stride, padding, dilation=1, out_dtype="float32")
+    out = topi.nn.relu(conv + bias)
+    return [data, kernel, bias, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create a search task for the last convolution layer in the resnet.
+
+target = tvm.target.Target("cuda")
+
+# the last layer in resnet
+N, H, W, CO, CI, KH, KW, strides, padding = 1, 7, 7, 512, 512, 3, 3, (1, 1), (1, 1)
+task = auto_scheduler.create_task(conv2d_layer, (N, H, W, CO, CI, KH, KW, strides, padding), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler. These parameters
+# mainly specify how we do the measurement during the search and auto-tuning.
+#
+# * `measure_ctx` launches a different process for measurement. This
+#   provides an isolation. It can protect the master process from GPU crashes
+#   happended during measurement and avoid other runtime conflicts.
+# * `min_repeat_ms` defines the minimum duration of one "repeat" in every measurement.
+#   This can warmup the GPU, which is necessary to get accurate measurement results.

Review comment:
       It would be better to mention the recommended value (i.e., >300) on GPU.

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -161,13 +178,16 @@ def resume_search(task, log_file):
 # .. note::
 #   We cannot run the line above because of the conflict between
 #   python's multiprocessing and tvm's thread pool.
-#   After running a tvm generated binary (L112), the python's multiprocessing
-#   library will hang forever.
-#   You have to make sure that you don't run any tvm generated binaries before
-#   calling ansor's search. To run the L156 above, you should comment out L112-114.
+#   After running a tvm generated binary the python's multiprocessing library
+#   will hang forever. You have to make sure that you don't run any tvm
+#   generated binaries before calling auot-scheduler's search.
+#   To run the function above, you should comment out all code in
+#   "Check correctness and evaluate performance" section.
 #
 #   You should be careful about this problem in your applications.
 #   There are other workarounds for this problem.
 #   For example, you can start a new thread/process (with the builtin python library
 #   threading or multiprocessing) and run the tvm binaries in the new thread/process.
 #   This provides an isolation and avoids the conflict in the main thread/process.
+#   You can also use :any:`auto_scheduler.measure.LocalRPCMeasureContext` for auto-scheduler,
+#   as shown in the GPU tutorial (:ref:`auto-scheduler-conv-gpu`).

Review comment:
       Intuitively, if there's no obvious performance impact, we should use RPC runner on both CPU and GPU, so it'd better to mention why we didn't use it in this tutorial.

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -59,6 +59,9 @@ def matmul_add(N, L, M, dtype):
 # Create the search task
 # ^^^^^^^^^^^^^^^^^^^^^^
 # We then create a search task with N=L=M=128 and dtype="float32"
+# If your machine supports avx instructions, you can
+# - replace "llvm" below with "llvm -mcpu=core-avx2" to enable AVX2
+# - replace "llvm" belwo with "llvm -mcpu=skylake-avx512" to enable AVX-512

Review comment:
       ```suggestion
   # - replace "llvm" below with "llvm -mcpu=skylake-avx512" to enable AVX-512
   ```

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -93,25 +96,38 @@ def matmul_add(N, L, M, dtype):
 ######################################################################
 # We can lower the schedule to see the IR after auto-scheduling.
 # The auto-scheduler correctly performs optimizations including multi-level tiling,
-# parallelization, vectorization, unrolling and fusion.
+# parallelization, vectorization, unrolling and operator fusion.
 
 print(tvm.lower(sch, args, simple_mode=True))
 
 ######################################################################
-# Check correctness
-# ^^^^^^^^^^^^^^^^^
-# We build the binary and check its correctness
+# Check correctness and evaluate performance
+# ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+# We build the binary and check its correctness and performance.
 
 func = tvm.build(sch, args)
 a_np = np.random.uniform(size=(128, 128)).astype(np.float32)
 b_np = np.random.uniform(size=(128, 128)).astype(np.float32)
 c_np = np.random.uniform(size=(128, 128)).astype(np.float32)
-d_np = a_np.dot(b_np) + c_np
-
-d_tvm = tvm.nd.empty(d_np.shape)
-func(tvm.nd.array(a_np), tvm.nd.array(b_np), tvm.nd.array(c_np), d_tvm)
-
-tvm.testing.assert_allclose(d_np, d_tvm.asnumpy(), rtol=1e-3)
+out_np = a_np.dot(b_np) + c_np
+
+ctx = tvm.cpu()
+a_tvm = tvm.nd.array(a_np, ctx=ctx)
+b_tvm = tvm.nd.array(b_np, ctx=ctx)
+c_tvm = tvm.nd.array(c_np, ctx=ctx)
+out_tvm = tvm.nd.empty(out_np.shape, ctx=ctx)
+func(a_tvm, b_tvm, c_tvm, out_tvm)
+
+# Check results
+tvm.testing.assert_allclose(out_np, out_tvm.asnumpy(), rtol=1e-3)
+
+# Evaluate execution time.
+evaluator = func.time_evaluator(func.entry_name, ctx, min_repeat_ms=500)
+print(
+    "Execution time of this operator: %.3f ms"
+    % (evaluator(a_tvm, b_tvm, c_tvm, out_tvm).mean * 1000)

Review comment:
       Should we use median instead of mean per offline discussion (same as in the GPU tutorial)?




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