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/04/24 22:36:45 UTC

[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5417: [RUNTIME][uTVM] AutoTVM + uTVM for Cortex-M7

weberlo commented on a change in pull request #5417:
URL: https://github.com/apache/incubator-tvm/pull/5417#discussion_r414765031



##########
File path: python/tvm/autotvm/tuner/tuner.py
##########
@@ -150,7 +150,15 @@ def tune(self, n_trial, measure_option, early_stopping=None, callbacks=(), si_pr
                              i + k + 1, si_prefix, format_si_prefix(flops, si_prefix),
                              format_si_prefix(self.best_flops, si_prefix), res, config)
 
-            i += len(results)
+            num_successes = 0
+            for result in results:
+                if isinstance(result.costs[0], float):
+                    num_successes += 1
+            if num_successes != len(results):
+                logger.debug('not counting %d failures towards trial count',
+                             len(results) - num_successes)
+            i += num_successes
+

Review comment:
       we should probably remove this for the time being. I added this when µTVM autotuning support was still pretty unstable, because there were _a lot_ of failed trials

##########
File path: python/tvm/contrib/binutil.py
##########
@@ -220,17 +228,32 @@ def tvm_callback_relocate_binary(
         stack_pointer_init=stack_pointer_init)
 
     tmp_dir = util.tempdir()
-    rel_obj_path = tmp_dir.relpath("relocated.obj")
-    rel_ld_script_path = tmp_dir.relpath("relocated.lds")
-    with open(rel_ld_script_path, "w") as f:
+    rel_obj_path = tmp_dir.relpath('relocated.obj')
+    rel_ld_script_path = tmp_dir.relpath('relocate.lds')
+    with open(rel_ld_script_path, 'w') as f:
         f.write(ld_script_contents)
     run_cmd([
-        "{}ld".format(toolchain_prefix),
+        '{}ld'.format(toolchain_prefix),
         binary_path,
-        "-T", rel_ld_script_path,
-        "-o", rel_obj_path])
-    with open(rel_obj_path, "rb") as f:
+        '-T', rel_ld_script_path,
+        '-o', rel_obj_path])
+
+    with open(rel_obj_path, 'rb') as f:
         rel_bin = bytearray(f.read())
+
+    gdb_init_dir = os.environ.get('MICRO_GDB_INIT_DIR')
+    if gdb_init_dir is not None:
+        gdb_init_path = f'{gdb_init_dir}/.gdbinit'
+        with open(gdb_init_path, 'r') as f:
+            gdbinit_contents = f.read().split('\n')
+        new_contents = []
+        for line in gdbinit_contents:
+            new_contents.append(line)
+            if line.startswith('target'):
+                new_contents.append(f'add-symbol-file {rel_obj_path}')
+        with open(gdb_init_path, 'w') as f:
+            f.write('\n'.join(new_contents))

Review comment:
       It might be worth splitting these lines into a separate µTVM debugging tools PR

##########
File path: python/tvm/micro/device/host.py
##########
@@ -38,59 +52,65 @@ def create_micro_lib(obj_path, src_path, lib_type, options=None):
 
     options : Optional[List[str]]
         additional options to pass to GCC
+
+    lib_src_paths : Optional[List[str]]
+        paths to additional source files to be compiled into the library
     """
     if options is None:
         options = []
-    if sys.maxsize > 2**32 and sys.platform.startswith("linux"):
-        options += ["-mcmodel=large"]
+    else:
+        options = list(options)
+    # Cannot increase optimization level on host due to code loading method.
+    options.append('-O0')

Review comment:
       `-Os` (and maybe `-O1`) work. it's just `-O2` that's been causing problems on the host

##########
File path: python/tvm/contrib/debugger/debug_runtime.py
##########
@@ -181,6 +181,7 @@ def _run_debug(self):
         """
         self.debug_datum._time_list = [
             [float(t) * 1e-6] for t in self.run_individual(10, 1, 1)
+            #[float(t) * 1e-6] for t in self.run_individual(1, 1, 1)

Review comment:
       remove

##########
File path: src/runtime/micro/host_driven/utvm_runtime.c
##########
@@ -34,89 +34,148 @@ extern "C" {
 
 #include "utvm_runtime.h"
 
-// Task pointers must be patched before calling a function.
-UTVMTask utvm_task = {
-    .func = NULL,
-    .arg_values = NULL,
-    .arg_type_codes = NULL,
-    .num_args = 0,
-};
-
-size_t utvm_word_size = 0;  // NOLINT(*)
+// TODO(areusch): move defines into header
+#define TASK_QUEUE_SIZE 20
+volatile UTVMTask utvm_tasks[TASK_QUEUE_SIZE] = { };
+volatile uint32_t utvm_num_tasks = 0;
+volatile uint32_t utvm_task_times[TASK_QUEUE_SIZE] = { };
 
 // These pointers are patched at load time to point to the workspace section.
-char* utvm_workspace_start = NULL;  // NOLINT(*)
-char* utvm_workspace_end = NULL;    // NOLINT(*)
-char* utvm_workspace_curr = NULL;   // NOLINT(*)
+volatile char* utvm_workspace_start = NULL;  // NOLINT(*)
+volatile char* utvm_workspace_end = NULL;    // NOLINT(*)
+volatile char* utvm_workspace_curr = NULL;   // NOLINT(*)
+#define MAX_WS_ALLOCS 10
+volatile char* utvm_alloc_ends[MAX_WS_ALLOCS] = {};  // NOLINT(*)
+volatile uint32_t utvm_alloc_idx = 0;
 // Keep track of how many active allocations there are on the workspace.
-size_t utvm_num_active_allocs = 0;
+volatile uint32_t utvm_num_active_allocs = 0;
+
+volatile uint32_t utvm_word_size = 0;
 
-const char* utvm_last_error = NULL;  // NOLINT(*)
-int32_t utvm_return_code = 0;        // NOLINT(*)
+volatile int32_t utvm_last_error = 0;  // NOLINT(*)
 
-uint32_t utvm_task_time = 0;
+volatile uint32_t utvm_done = 0;
 
 // Gets called by UTVMInit, after device-specific initialization is finished.
 void UTVMMain() {
+  utvm_done = 0;
+  // loss of precision should be fine here, since we only care about the lower bits
+  if (((uint32_t) utvm_workspace_start) % utvm_word_size) {
+    utvm_last_error = UTVM_ERR_WS_UNALIGNED_START;
+    UTVMDone();
+    return;
+  }
   utvm_workspace_curr = utvm_workspace_start;
   utvm_num_active_allocs = 0;
-  utvm_last_error = NULL;  // NOLINT(*)
-  utvm_return_code = 0;
-  utvm_task_time = 0;
-  UTVMTimerReset();
-  int32_t err = UTVMTimerStart();
-  if (err < 0) {
-    utvm_return_code = err;
-    UTVMDone();
+  utvm_alloc_idx = 0;
+  utvm_last_error = UTVM_ERR_NOT_FINISHED;
+  for (uint32_t i = 0; i < utvm_num_tasks; i++) {
+    int32_t err = UTVM_ERR_OK;
+    utvm_task_times[i] = 0;
+    err = UTVMTimerStart();
+    if (err < 0) {
+      utvm_last_error = err;
+      UTVMDone();
+      return;
+    }
+    err = utvm_tasks[i].func(
+        (void*) utvm_tasks[i].arg_values,      // NOLINT(*)
+        (void*) utvm_tasks[i].arg_type_codes,  // NOLINT(*)
+        utvm_tasks[i].num_args);
+    if (err < 0) {
+      UTVMDone();
+      return;
+    }
+    utvm_task_times[i] = UTVMTimerStop(&err);
+    if (err < 0) {
+      utvm_last_error = err;
+      UTVMDone();
+      return;
+    }
+  }
+  if (utvm_last_error == UTVM_ERR_NOT_FINISHED) {
+    utvm_last_error = UTVM_ERR_OK;
   }
-  utvm_return_code = utvm_task.func(
-          (void*) utvm_task.arg_values,      // NOLINT(*)
-          (void*) utvm_task.arg_type_codes,  // NOLINT(*)
-          utvm_task.num_args);
-  UTVMTimerStop();
-  utvm_task_time = UTVMTimerRead();
   UTVMDone();
 }
 
 // We use a dummy function to signal execution is finished for device
 // backends which require breakpoints.
-void UTVMDone() { }
+void __attribute__((noinline)) UTVMDone() {
+  utvm_done = 1;
+}
+
+#define ALIGNED_UP(x, word_size) \
+  ((((word_size) - (((uintptr_t) (x)) % (word_size))) % (word_size)) + (x))
 
 void* TVMBackendAllocWorkspace(int device_type, int device_id, uint64_t size,
                                int dtype_code_hint, int dtype_bits_hint) {
-  // Align up to 8 bytes.
-  utvm_workspace_curr +=
-    (utvm_word_size - ((uintptr_t) utvm_workspace_curr % utvm_word_size)) % utvm_word_size;  // NOLINT(*)
-  if (utvm_workspace_curr + size > utvm_workspace_end) {
+  if (size == 0) {
+    utvm_last_error = UTVM_ERR_WS_ZERO_SIZE_ALLOC;
+    return NULL;
+  }
+  size_t alloc_requested_bytes = size;
+  size_t alloc_size_words = (alloc_requested_bytes + utvm_word_size - 1) / utvm_word_size;
+  size_t alloc_size_bytes = alloc_size_words * utvm_word_size;
+
+  // Align up to the target word size.
+  if (utvm_workspace_curr + alloc_size_bytes > utvm_workspace_end) {
     // Out of space in workspace.
+    utvm_last_error = UTVM_ERR_WS_OUT_OF_SPACE;
+    return NULL;
+  }
+  if (utvm_alloc_idx == MAX_WS_ALLOCS - 1) {
+    // Exceeded number of allocs we can keep track of.
+    utvm_last_error = UTVM_ERR_WS_TOO_MANY_ALLOCS;
     return NULL;
   }
   void* ret_ptr = (void*) utvm_workspace_curr;  // NOLINT(*)
-  utvm_workspace_curr += size;
+  utvm_workspace_curr = utvm_workspace_curr + alloc_size_bytes;
+  // store the *end* of the alloc, so we can restore the WS pointer when freeing
+  utvm_alloc_ends[utvm_alloc_idx] = utvm_workspace_curr;
+  utvm_alloc_idx++;
   utvm_num_active_allocs++;
   return ret_ptr;
 }
 
 int TVMBackendFreeWorkspace(int device_type, int device_id, void* ptr) {
-  utvm_num_active_allocs--;
-  if (utvm_num_active_allocs < 0) {
+  // TODO(areusch): add dev type check
+  if (utvm_num_active_allocs == 0) {
     TVMAPISetLastError("free called with no active workspace allocations");
     // Reset allocations and workspace (for future task executions).
     utvm_num_active_allocs = 0;
     utvm_workspace_curr = utvm_workspace_start;
+    utvm_last_error = UTVM_ERR_WS_DOUBLE_FREE;
     return -1;
-  } else if (utvm_num_active_allocs == 0) {
-    // No more allocations.  Reset workspace.
-    utvm_workspace_curr = utvm_workspace_start;
-    return 0;
   } else {
+    utvm_num_active_allocs--;
+    if (ptr == utvm_workspace_start) {
+      // it's the first allocation
+      utvm_alloc_ends[0] = NULL;
+    } else {
+      // TODO(areusch): reverse loop iteration since usually it's the last alloc being freed

Review comment:
       i think this TODO is fulfilled

##########
File path: python/tvm/relay/_parser.py
##########
@@ -340,7 +342,10 @@ def visitLocalVar(self, ctx):
         return local_var
 
     def visitGraphVar(self, ctx):
-        return self.graph_expr[int(ctx.NAT().getText())]
+        graph_var_idx = int(ctx.NAT().getText())
+        if graph_var_idx >= len(self.graph_expr):
+            raise ParseError(f"graph var `%{graph_var_idx}` is unbound")
+        return self.graph_expr[graph_var_idx]

Review comment:
       this change is a good quality-of-life improvement for the Relay parser, but should likely be a separate PR

##########
File path: python/tvm/relay/op/strategy/arm_cpu.py
##########
@@ -267,7 +281,7 @@ def bitserial_conv2d_strategy_arm_cpu(attrs, inputs, out_type, target):
         raise ValueError("Data layout {} not supported.".format(layout))
     return strategy
 
-@bitserial_dense_strategy.register("arm_cpu")
+@bitserial_dense_strategy.register(["arm_cpu", "micro_dev"])

Review comment:
       do the bitserial schedules work on cortex-m?
   also, are we punting the device vs. runtime problem (e.g., `device=arm_cpu` but `runtime=micro`) to a future PR?

##########
File path: python/tvm/target/arm_isa.py
##########
@@ -0,0 +1,34 @@
+# 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.
+"""Defines functions to analyze availble opcodes in the ARM ISA."""

Review comment:
       ```suggestion
   """Defines functions to analyze available opcodes in the ARM ISA."""
   ```

##########
File path: python/tvm/micro/base.py
##########
@@ -133,44 +152,91 @@ def __exit__(self, exc_type, exc_value, exc_traceback):
         self._exit()
 
 
-def create_micro_mod(c_mod, dev_config):
+def _calc_max_workspace_usage(src):
+    # TODO factor in alignment to the calculation (alloc sizes will be aligned up to the word size)
+    alloc_re = re.compile(
+        r'.*\* ?(.+) = (\(.+\))? TVMBackendAllocWorkspace\(.+, .+, \(uint64_t\)(.+), .+, .+\).*')
+    free_re = re.compile(r'.*if \(TVMBackendFreeWorkspace\(.+, .+, (\(void\*\))? (.+)\) != 0\) {.*')
+    max_usage = 0
+    alloc_map = {}
+    for line in src.split('\n'):
+        if line.strip().startswith('//'):
+            continue
+        match = alloc_re.match(line)
+        if match is not None:
+            alloc_map[match.group(1)] = int(match.group(3))
+            max_usage = max(max_usage, sum(alloc_map.values()))
+        else:
+            match = free_re.match(line)
+            if match is not None:
+                print(alloc_map)
+                del alloc_map[match.group(2)]
+    return max_usage
+
+

Review comment:
       this is for sure a hacky way to calculate the memory footprint of workspace allocations.
   in a followup PR, we should move this calculation further upstream and instead use a visitor to find workspace allocs in the AST.
   in the meantime, let's just make sure it doesn't ever crash when `src` doesn't match the format expected by the regexes.

##########
File path: src/ir/error.cc
##########
@@ -62,16 +62,21 @@ void ErrorReporter::RenderErrors(const IRModule& module, bool use_color) {
 
     CHECK(has_errs != this->node_to_error_.end());
 
-    const auto& error_indicies = has_errs->second;
+    const auto& error_indices = has_errs->second;
 
     std::stringstream err_msg;
 
-    err_msg << rang::fg::red;
-    err_msg << " ";
-    for (auto index : error_indicies) {
-      err_msg << this->errors_[index].what() << "; ";
+    if (error_indices.size() != 0) {
+      err_msg << rang::fg::red;
+      err_msg << " ";
+      // the errors are in reverse order, so print them with a reversed iteration
+      err_msg << this->errors_[error_indices[error_indices.size()-1]].what();
+      for (int i = error_indices.size() - 2; i >= 0; i--) {
+        size_t err_idx = error_indices[i];
+        err_msg << "; " << this->errors_[err_idx].what();
+      }
+      err_msg << rang::fg::reset;

Review comment:
       this is another change that improves error messages (for whatever reason, they're displayed in the reverse order of how you want to read them), but should also be a separate PR.

##########
File path: src/runtime/micro/host_driven/utvm_runtime.c
##########
@@ -34,89 +34,148 @@ extern "C" {
 
 #include "utvm_runtime.h"
 
-// Task pointers must be patched before calling a function.
-UTVMTask utvm_task = {
-    .func = NULL,
-    .arg_values = NULL,
-    .arg_type_codes = NULL,
-    .num_args = 0,
-};
-
-size_t utvm_word_size = 0;  // NOLINT(*)
+// TODO(areusch): move defines into header
+#define TASK_QUEUE_SIZE 20
+volatile UTVMTask utvm_tasks[TASK_QUEUE_SIZE] = { };
+volatile uint32_t utvm_num_tasks = 0;
+volatile uint32_t utvm_task_times[TASK_QUEUE_SIZE] = { };
 
 // These pointers are patched at load time to point to the workspace section.
-char* utvm_workspace_start = NULL;  // NOLINT(*)
-char* utvm_workspace_end = NULL;    // NOLINT(*)
-char* utvm_workspace_curr = NULL;   // NOLINT(*)
+volatile char* utvm_workspace_start = NULL;  // NOLINT(*)
+volatile char* utvm_workspace_end = NULL;    // NOLINT(*)
+volatile char* utvm_workspace_curr = NULL;   // NOLINT(*)
+#define MAX_WS_ALLOCS 10
+volatile char* utvm_alloc_ends[MAX_WS_ALLOCS] = {};  // NOLINT(*)

Review comment:
       not sure if we want to make this configurable as well. it's definitely not as problematic as `TASK_QUEUE_SIZE`

##########
File path: src/runtime/micro/host_driven/utvm_runtime.c
##########
@@ -34,89 +34,148 @@ extern "C" {
 
 #include "utvm_runtime.h"
 
-// Task pointers must be patched before calling a function.
-UTVMTask utvm_task = {
-    .func = NULL,
-    .arg_values = NULL,
-    .arg_type_codes = NULL,
-    .num_args = 0,
-};
-
-size_t utvm_word_size = 0;  // NOLINT(*)
+// TODO(areusch): move defines into header
+#define TASK_QUEUE_SIZE 20
+volatile UTVMTask utvm_tasks[TASK_QUEUE_SIZE] = { };
+volatile uint32_t utvm_num_tasks = 0;
+volatile uint32_t utvm_task_times[TASK_QUEUE_SIZE] = { };
 
 // These pointers are patched at load time to point to the workspace section.
-char* utvm_workspace_start = NULL;  // NOLINT(*)
-char* utvm_workspace_end = NULL;    // NOLINT(*)
-char* utvm_workspace_curr = NULL;   // NOLINT(*)
+volatile char* utvm_workspace_start = NULL;  // NOLINT(*)
+volatile char* utvm_workspace_end = NULL;    // NOLINT(*)
+volatile char* utvm_workspace_curr = NULL;   // NOLINT(*)
+#define MAX_WS_ALLOCS 10
+volatile char* utvm_alloc_ends[MAX_WS_ALLOCS] = {};  // NOLINT(*)
+volatile uint32_t utvm_alloc_idx = 0;
 // Keep track of how many active allocations there are on the workspace.
-size_t utvm_num_active_allocs = 0;
+volatile uint32_t utvm_num_active_allocs = 0;
+
+volatile uint32_t utvm_word_size = 0;
 
-const char* utvm_last_error = NULL;  // NOLINT(*)
-int32_t utvm_return_code = 0;        // NOLINT(*)
+volatile int32_t utvm_last_error = 0;  // NOLINT(*)
 
-uint32_t utvm_task_time = 0;
+volatile uint32_t utvm_done = 0;
 
 // Gets called by UTVMInit, after device-specific initialization is finished.
 void UTVMMain() {
+  utvm_done = 0;
+  // loss of precision should be fine here, since we only care about the lower bits
+  if (((uint32_t) utvm_workspace_start) % utvm_word_size) {

Review comment:
       downcasting to a potentially smaller integral type shouldn't trigger undefined behavior, right?

##########
File path: src/runtime/micro/micro_session.cc
##########
@@ -103,102 +108,137 @@ MicroSession::MicroSession(
     low_level_device_ = HostLowLevelDeviceCreate(memory_size, &base_addr);
     CHECK_EQ(reinterpret_cast<std::uintptr_t>(base_addr) % word_size_, 0)
       << "base address not aligned to " << word_size_ << " bytes";
-    DevPtr curr_addr = DevPtr(reinterpret_cast<std::uintptr_t>(base_addr));
+    TargetPtr curr_addr = TargetPtr(reinterpret_cast<std::uintptr_t>(base_addr));
 
-    section_allocators_[0] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
+    section_allocators_[0] = std::make_shared<MicroSectionAllocator>(
+      "text",
+      DevMemRegion {
       .start = curr_addr,
       .size = text_size,
-    }, word_size_);
+      }, word_size_);
     curr_addr += text_size;
-    section_allocators_[1] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = rodata_size,
-    }, word_size_);
+    section_allocators_[1] = std::make_shared<MicroSectionAllocator>(
+      "rodata",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = rodata_size,
+      }, word_size_);
     curr_addr += rodata_size;
-    section_allocators_[2] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = data_size,
-    }, word_size_);
+    section_allocators_[2] = std::make_shared<MicroSectionAllocator>(
+      "data",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = data_size,
+      }, word_size_);
     curr_addr += data_size;
-    section_allocators_[3] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = bss_size,
-    }, word_size_);
+    section_allocators_[3] = std::make_shared<MicroSectionAllocator>(
+      "bss",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = bss_size,
+      }, word_size_);
     curr_addr += bss_size;
-    section_allocators_[4] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = args_size,
-    }, word_size_);
+    section_allocators_[4] = std::make_shared<MicroSectionAllocator>(
+      "args",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = args_size,
+      }, word_size_);
     curr_addr += args_size;
-    section_allocators_[5] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = heap_size,
-    }, word_size_);
+    section_allocators_[5] = std::make_shared<MicroSectionAllocator>(
+      "heap",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = heap_size,
+      }, word_size_);
     curr_addr += heap_size;
-    section_allocators_[6] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = workspace_size,
-    }, word_size_);
+    section_allocators_[6] = std::make_shared<MicroSectionAllocator>(
+      "workspace",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = workspace_size,
+      }, word_size_);
     curr_addr += workspace_size;
-    section_allocators_[7] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = curr_addr,
-      .size = stack_size,
-    }, word_size_);
+    section_allocators_[7] = std::make_shared<MicroSectionAllocator>(
+      "stack",
+      DevMemRegion {
+        .start = curr_addr,
+        .size = stack_size,
+      }, word_size_);
     curr_addr += stack_size;
   } else if (comms_method == "openocd") {
     low_level_device_ = OpenOCDLowLevelDeviceCreate(server_addr, port);
-    section_allocators_[0] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(text_start),
-      .size = text_size,
-    }, word_size_);
-    section_allocators_[1] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(rodata_start),
-      .size = rodata_size,
-    }, word_size_);
-    section_allocators_[2] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(data_start),
-      .size = data_size,
-    }, word_size_);
-    section_allocators_[3] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(bss_start),
-      .size = bss_size,
-    }, word_size_);
-    section_allocators_[4] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(args_start),
-      .size = args_size,
-    }, word_size_);
-    section_allocators_[5] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(heap_start),
-      .size = heap_size,
-    }, word_size_);
-    section_allocators_[6] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(workspace_start),
-      .size = workspace_size,
-    }, word_size_);
-    section_allocators_[7] = std::make_shared<MicroSectionAllocator>(DevMemRegion {
-      .start = DevPtr(stack_start),
-      .size = stack_size,
-    }, word_size_);
+    section_allocators_[0] = std::make_shared<MicroSectionAllocator>(
+      "text",
+      DevMemRegion {
+        .start = TargetPtr(text_start),
+        .size = text_size,
+      }, word_size_);
+    section_allocators_[1] = std::make_shared<MicroSectionAllocator>(
+      "rodata",
+      DevMemRegion {
+        .start = TargetPtr(rodata_start),
+        .size = rodata_size,
+      }, word_size_);
+    section_allocators_[2] = std::make_shared<MicroSectionAllocator>(
+      "data",
+      DevMemRegion {
+        .start = TargetPtr(data_start),
+        .size = data_size,
+      }, word_size_);
+    section_allocators_[3] = std::make_shared<MicroSectionAllocator>(
+      "bss",
+      DevMemRegion {
+        .start = TargetPtr(bss_start),
+        .size = bss_size,
+      }, word_size_);
+    section_allocators_[4] = std::make_shared<MicroSectionAllocator>(
+      "args",
+      DevMemRegion {
+        .start = TargetPtr(args_start),
+        .size = args_size,
+      }, word_size_);
+    section_allocators_[5] = std::make_shared<MicroSectionAllocator>(
+      "heap",
+      DevMemRegion {
+        .start = TargetPtr(heap_start),
+        .size = heap_size,
+      }, word_size_);
+    section_allocators_[6] = std::make_shared<MicroSectionAllocator>(
+      "workspace",
+      DevMemRegion {
+        .start = TargetPtr(workspace_start),
+        .size = workspace_size,
+      }, word_size_);
+    section_allocators_[7] = std::make_shared<MicroSectionAllocator>(
+      "stack",
+      DevMemRegion {
+        .start = TargetPtr(stack_start),
+        .size = stack_size,
+      }, word_size_);

Review comment:
       this should be a loop

##########
File path: src/runtime/micro/host_driven/utvm_runtime.c
##########
@@ -34,89 +34,148 @@ extern "C" {
 
 #include "utvm_runtime.h"
 
-// Task pointers must be patched before calling a function.
-UTVMTask utvm_task = {
-    .func = NULL,
-    .arg_values = NULL,
-    .arg_type_codes = NULL,
-    .num_args = 0,
-};
-
-size_t utvm_word_size = 0;  // NOLINT(*)
+// TODO(areusch): move defines into header
+#define TASK_QUEUE_SIZE 20
+volatile UTVMTask utvm_tasks[TASK_QUEUE_SIZE] = { };
+volatile uint32_t utvm_num_tasks = 0;
+volatile uint32_t utvm_task_times[TASK_QUEUE_SIZE] = { };

Review comment:
       could you add another TODO to unify `TASK_QUEUE_SIZE` here and `MicroSession::kTaskQueueCapacity`? they both currently need to be set to the same value, which scares the livin' daylights outta me.
   
   a better setup could be to have the task queue size as a field in the device config. then `MicroSession` can be initted with that value, and while compiling the runtime, the queue size can be exposed as a preprocessor definition `TASK_QUEUE_SIZE`.

##########
File path: src/runtime/micro/micro_session.cc
##########
@@ -489,6 +629,16 @@ PackedFunc MicroSession::GetFunction(
     return PackedFunc([sptr_to_self](TVMArgs args, TVMRetValue* rv) {
       MicroSession::ExitWithScope();
     });
+    // TODO(weberlo): add a `clear_batch_timer` func
+  } else if (name == "get_last_batch_time") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      *rv = this->GetLastBatchTime();
+    });
+    // TODO(weberlo): remove this func
+  } else if (name == "get_last_batch_cycles") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      *rv = this->GetLastBatchCycles();
+    });

Review comment:
       perhaps we should rename these functions to `GetLastBatchHostTime` and `GetLastBatchDevTime`. I think having both would be of use, for example, if a user wants to verify their device timer impl with host timings, or if a device doesn't have a timer (i think this case is rare tho).
   
   we may also want to rethink the timing API, because resetting the batch time to 0 when `GetLastBatchTime` is called isn't very user-friendly.

##########
File path: topi/python/topi/arm_cpu/cortex_m7/micro_kernel/gemm.py
##########
@@ -0,0 +1,221 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, no-value-for-parameter
+"""Defines gemm intrinsics for SIMD matrix multiplication."""
+
+import random
+import string
+
+import tvm
+from tvm import te
+
+##########################
+# MxKxN MatMul Intrinsic #
+##########################
+
+# NOTE this is transposed matmul (A * B^T)
+def intrin_gemm_MxKxN(M, K, N, in_dtype, out_dtype):
+    """Defines a SIMD-accelerated transposed matmul."""
+    # we generate a unique ID for every intrinsic definition, to prevent name
+    # collisions in the generated source (e.g., if there are multiple operators
+    # in the same module that use the same intrinsic)
+    #
+    # TODO to cut down on memory usage, we should cache each intrinsic
+    # instantiation and include it only once, eliminating the need for unique
+    # IDs
+    UNIQ_ID_LEN = 8
+    uniq_id = ''.join(random.choices(string.ascii_uppercase, k=UNIQ_ID_LEN))
+
+    if isinstance(M, tvm.tir.IntImm):
+        M = M.value
+    if isinstance(K, tvm.tir.IntImm):
+        K = K.value
+    if isinstance(N, tvm.tir.IntImm):
+        N = N.value
+    assert K % 4 == 0
+    # TODO support more dtypes?
+    assert in_dtype == 'int8'
+    assert out_dtype == 'int32'
+    A = te.placeholder((M, K), name='a', dtype=in_dtype)
+    B = te.placeholder((N, K), name='b', dtype=in_dtype)
+    k = te.reduce_axis((0, K), name='k')
+    C = te.compute(
+        (M, N),
+        lambda i, j: te.sum(A[i, k].astype(out_dtype) * B[j, k].astype(out_dtype), axis=k),
+        name='c')
+    A_buf = tvm.tir.decl_buffer(
+        A.shape, A.dtype,
+        name="A",
+        offset_factor=1,
+        strides=[te.var("A_s"), 1])
+    B_buf = tvm.tir.decl_buffer(
+        B.shape, B.dtype,
+        name="B",
+        offset_factor=1,
+        strides=[te.var("B_s"), 1])
+    C_buf = tvm.tir.decl_buffer(
+        C.shape, C.dtype,
+        name="C",
+        offset_factor=1,
+        strides=[te.var("C_s"), 1])
+    def intrin_func(ins, outs):
+        aa, bb = ins
+        cc = outs[0]
+        def _reduce_update():
+            ib = tvm.tir.ir_builder.create()
+            ib.emit(tvm.tir.call_extern("int32", f"gemm_{M}x{K}x{N}_update_{uniq_id}",
+                                        aa.access_ptr("r"),
+                                        bb.access_ptr("r"),
+                                        cc.access_ptr("w"),
+                                        aa.strides[0],
+                                        bb.strides[0],
+                                        cc.strides[0]))
+            return ib.get()
+        def _reduce_reset():
+            ib = tvm.tir.ir_builder.create()
+            ib.emit(tvm.tir.call_extern("int32", f"gemm_{M}x{K}x{N}_reset_{uniq_id}",
+                                        cc.access_ptr("w"),
+                                        cc.strides[0]))
+            return ib.get()
+        def _body():
+            ib = tvm.tir.ir_builder.create()
+            # # NOTE we need the reset in the body for cases where the buffer
+            # # we're accumulating into is uninitialized (e.g., if it's the
+            # # result of a workspace allocation, because there are no guarantees
+            # # on the contents).
+            # ib.emit(tvm.tir.call_extern("int32", f"gemm_{M}x{K}x{N}_reset",
+            #                         cc.access_ptr("w"),
+            #                         cc.strides[0]))
+            # ib.emit(tvm.tir.call_extern("int32", f"gemm_{M}x{K}x{N}_update",
+            #                         aa.access_ptr("r"),
+            #                         bb.access_ptr("r"),
+            #                         cc.access_ptr("w"),
+            #                         aa.strides[0],
+            #                         bb.strides[0],
+            #                         cc.strides[0]))
+            ib.emit(tvm.tir.call_extern("int32", f"gemm_{M}x{K}x{N}_body_{uniq_id}",
+                                        aa.access_ptr("r"),
+                                        bb.access_ptr("r"),
+                                        cc.access_ptr("w"),
+                                        aa.strides[0],
+                                        bb.strides[0],
+                                        cc.strides[0]))
+            return ib.get()
+        return _body(), _reduce_reset(), _reduce_update()
+    with tvm.target.build_config(offset_factor=1):
+        intrin_decl = te.decl_tensor_intrin(
+            C.op, intrin_func, binds={A: A_buf, B: B_buf, C: C_buf})
+        return intrin_decl, uniq_id
+
+
+def gemm_MxKxN_impl(M, K, N, uniq_id):
+    """Emit C code for gemm impl."""
+    # TODO are there any SIMD tricks to zero out arrays quickly?

Review comment:
       usernames

##########
File path: src/runtime/micro/openocd_low_level_device.cc
##########
@@ -210,9 +210,9 @@ class OpenOCDLowLevelDevice final : public LowLevelDevice {
   // NOTE: OpenOCD will call any request larger than this constant an "absurd
   // request".
   /*! \brief maximum number of bytes allowed in a single memory transfer */
-  static const constexpr ssize_t kMemTransferLimit = 64000;
+  static const constexpr ssize_t kMemTransferLimit = 8000;

Review comment:
       i'm curious what openocd version you're running, because it seems like the standard for an "absurd request" is 64k ([line 4274](http://openocd.org/doc/doxygen/html/target_2target_8c_source.html))

##########
File path: tests/python/unittest/test_runtime_micro.py
##########
@@ -25,8 +25,25 @@
 from tvm.micro import create_micro_mod
 from tvm.relay.testing import resnet
 
-# Use the host emulated micro device.
-DEV_CONFIG = micro.device.host.default_config()
+# # Use the host emulated micro device.
+DEV_CONFIG_A = micro.device.host.generate_config()
+DEV_CONFIG_B = micro.device.host.generate_config()
+TARGET = 'c -device=micro_dev'

Review comment:
       these can be re-collapsed into a single `DEV_CONFIG`. I separated them for prototyping, because you need separate server ports for physical devices

##########
File path: topi/python/topi/arm_cpu/conv2d_spatial_pack.py
##########
@@ -173,7 +173,7 @@ def schedule_conv2d_spatial_pack_nchw(cfg, s, data_vec, kernel_vec,
                                  axis_lens=[cfg['tile_oh'].size[-1],
                                             cfg['tile_ow'].size[-1],
                                             cfg['tile_co'].size[-1]],
-                                 max_unroll=16,
+                                 max_unroll=None,

Review comment:
       we should probs reset the `max_unroll` to 16

##########
File path: src/runtime/micro/micro_session.cc
##########
@@ -404,39 +509,74 @@ DevPtr MicroSession::EncoderAppend(TargetDataLayoutEncoder* encoder, const DLTen
   return tvm_arr_slot.start_addr();
 }
 
+// TODO(weberlo): switch over entirely to error codes that expand to error
+// messages on the host side.

Review comment:
       this todo has been fulfilled

##########
File path: topi/python/topi/arm_cpu/cortex_m7/micro_kernel/gemm.py
##########
@@ -0,0 +1,221 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, no-value-for-parameter
+"""Defines gemm intrinsics for SIMD matrix multiplication."""
+
+import random
+import string
+
+import tvm
+from tvm import te
+
+##########################
+# MxKxN MatMul Intrinsic #
+##########################
+
+# NOTE this is transposed matmul (A * B^T)
+def intrin_gemm_MxKxN(M, K, N, in_dtype, out_dtype):
+    """Defines a SIMD-accelerated transposed matmul."""
+    # we generate a unique ID for every intrinsic definition, to prevent name
+    # collisions in the generated source (e.g., if there are multiple operators
+    # in the same module that use the same intrinsic)
+    #
+    # TODO to cut down on memory usage, we should cache each intrinsic
+    # instantiation and include it only once, eliminating the need for unique
+    # IDs
+    UNIQ_ID_LEN = 8
+    uniq_id = ''.join(random.choices(string.ascii_uppercase, k=UNIQ_ID_LEN))
+
+    if isinstance(M, tvm.tir.IntImm):
+        M = M.value
+    if isinstance(K, tvm.tir.IntImm):
+        K = K.value
+    if isinstance(N, tvm.tir.IntImm):
+        N = N.value
+    assert K % 4 == 0
+    # TODO support more dtypes?

Review comment:
       add usernames

##########
File path: src/runtime/micro/host_driven/utvm_device_dylib_redirect.c
##########
@@ -32,10 +32,10 @@ extern "C" {
 #include <stdint.h>
 #include <stddef.h>
 
-void *(*TVMBackendAllocWorkspace_)(int, int, uint64_t, int, int) =
-    (void *(*)(int, int, uint64_t, int, int)) NULL;
-int (*TVMBackendFreeWorkspace_)(int, int, void*) = (int (*)(int, int, void*)) NULL;
-void (*TVMAPISetLastError_)(const char*) = (void (*)(const char*)) NULL;
+// TODO(areusch): compiler errors say volatile qualifier is discarded. should we just get rid of em?

Review comment:
       add back `weberlo` to the TODOs you replaced with `areusch` (e.g., `TODO (weberlo, areusch)`), so I can still grep for them




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