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 2022/02/23 15:26:12 UTC

[GitHub] [tvm] kparzysz-quic opened a new pull request #10361: [Hexagon] RPC server/client for simulator

kparzysz-quic opened a new pull request #10361:
URL: https://github.com/apache/tvm/pull/10361


   This is the C++ code for running Hexagon code on simulator via the RPC mechanism. It is intended to be integrated into the current `HexagonLauncher`, although the integration will require further changes to the launcher python code.
   
   The final goal is to be able to run the same file.py on either hardware or simulator without needing to edit the python file, but simply by changing the configuration of the execution platform (i.e. something like --execute-on=simulator as a command line or in an environment variable). The exact details are still to be determined.
   


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

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

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



[GitHub] [tvm] masahi commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

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



##########
File path: src/runtime/hexagon/rpc/simulator/session.cc
##########
@@ -0,0 +1,1322 @@
+/*
+ * 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.
+ */
+
+#include <HexagonWrapper.h>
+#include <dmlc/optional.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+// POSIX includes
+#include <dirent.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <deque>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <utility>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "hexagon_sim_proto.h"
+
+#define CHECKED_CALL(func, ...)                                               \
+  do {                                                                        \
+    HEXAPI_Status s = sim_->func(__VA_ARGS__);                                \
+    ICHECK_EQ(s, HEX_STAT_SUCCESS)                                            \
+        << self_name_ << ": " #func " failed with code " << Status_{s}.str(); \
+  } while (false)
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+using string_list = std::deque<std::string>;
+
+namespace detail {
+
+// An "Optional" class, originally a replacement for llvm::Optional, then an
+// extension of dmlc::optional to make it compatible with C++17's std::optional.
+template <typename T>
+struct Optional : public dmlc::optional<T> {
+  using dmlc::optional<T>::optional;
+  using dmlc::optional<T>::operator=;
+  Optional(const T& val) : dmlc::optional<T>(val) {}  // NOLINT(*)
+
+  T* operator->() { return &this->operator*(); }
+  const T* operator->() const { return &this->operator*(); }
+};
+
+// Replacement for llvm::StringSwitch.
+template <typename T>
+class StringSwitch {
+ public:
+  explicit StringSwitch(const std::string& key) : key(key) {}
+  operator T() const {
+    auto f = map.find(key);
+    if (f != map.end()) {
+      return f->second;
+    }
+    ICHECK(static_cast<bool>(def_val)) << "default value not set";
+    return *def_val;
+  }
+  StringSwitch& Case(const std::string& key, T val) {
+    map.insert(std::make_pair(key, val));
+    return *this;
+  }
+  StringSwitch& Default(T val) {
+    ICHECK(!static_cast<bool>(def_val)) << "default value already set";
+    def_val = val;
+    return *this;
+  }
+
+ private:
+  const std::string key;
+  std::map<std::string, T> map;
+  Optional<T> def_val;
+};
+
+using MaybeString = Optional<std::string>;
+
+MaybeString front(const string_list& deq) {
+  return !deq.empty() ? MaybeString(deq.front()) : MaybeString();
+}
+
+MaybeString pop_front(string_list& deq) {  // NOLINT(*)
+  if (deq.empty()) return MaybeString();
+  std::string f = deq.front();
+  deq.pop_front();
+  return MaybeString(f);
+}
+
+// Functions used when parsing the argument string.
+
+Optional<int64_t> to_int(const MaybeString& str) {
+  auto none = Optional<int64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      int64_t val = std::stoll(*str, &pos, 0);
+      return pos == str->size() ? Optional<int64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<uint64_t> to_uint(const MaybeString& str) {
+  auto none = Optional<uint64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      uint64_t val = std::stoull(*str, &pos, 0);
+      return pos == str->size() ? Optional<uint64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<float> to_float(const MaybeString& str) {
+  auto none = Optional<float>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      float val = std::stof(*str, &pos);
+      return pos == str->size() ? Optional<float>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<bool> to_bool(const MaybeString& str) {
+  auto none = Optional<bool>();
+  if (auto num = to_int(str)) {
+    if (*num == 0) return false;
+    if (*num == 1) return true;
+    return none;
+  }
+  if (str) {
+    if (*str == "true" || *str == "TRUE") return true;
+    if (*str == "false" || *str == "FALSE") return false;
+  }
+  return none;
+}
+
+template <typename T>
+using MaybeRange = Optional<std::pair<T, T>>;
+
+template <typename T, Optional<T> Parse(const MaybeString&)>
+MaybeRange<T> to_range(const MaybeString& str) {
+  auto none = MaybeRange<T>();
+  if (str && !str->empty()) {
+    auto n = str->find('-', 1);
+    if (n != std::string::npos) {
+      auto begin = Parse(str->substr(0, n));
+      auto end = Parse(str->substr(n + 1, str->size() - n - 1));
+      if (begin && end) {
+        return std::make_pair(*begin, *end);
+      }
+    }
+  }
+  return none;
+}
+
+}  // namespace detail
+
+class SimulatorRPCChannel final : public RPCChannel {
+ public:
+  SimulatorRPCChannel(std::string args);
+  ~SimulatorRPCChannel() final;
+  size_t Send(const void* data, size_t size) final;
+  size_t Recv(void* data, size_t size) final;
+
+ private:
+  struct Status_ {
+    HEXAPI_Status s;
+    std::string str() const;
+  };
+  struct Core_ {
+    HEXAPI_CoreState s;
+    std::string str() const;
+  };
+  struct Cpu_ {
+    HEXAPI_Cpu c;
+    std::string str() const;
+  };
+  struct SDKInfo_ {
+    SDKInfo_(const std::string& sdk_root, const std::string& cpu);
+    std::string root;
+    std::string qurt_root;  // sdk_root/rtos/qurt/computevNN.
+    std::string runelf;     // Path to runelf.pbn.
+    std::string runmain;    // Path to run_main_on_hexagon.
+  };
+
+  Message SendMsg(Message msg);
+  Message SendMsg(uint32_t code, uint32_t len, uint32_t va);
+  void ReadFromProcess(void* host_dst, HEX_VA_t src, size_t len);
+  void WriteToProcess(HEX_VA_t dst, const void* host_src, size_t len);
+
+  static HEX_8u_t PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV, HEX_8u_t RttV,
+                                       HEX_8u_t RxxV, HEX_1u_t imm);
+
+  detail::Optional<HEXAPI_Cpu> GetCPU(const detail::MaybeString& cpu_str);
+
+  // File name templates for mkstemps.
+#define SUFFIX ".cfg"
+  static constexpr int template_length_ = strlen("temp-xxxx-XXXXXX" SUFFIX) + 1;
+  char osam_file_[template_length_] = "temp-osam-XXXXXX" SUFFIX;
+  char cosim_file_[template_length_] = "temp-q6ss-XXXXXX" SUFFIX;
+  const int suffix_len_ = strlen(SUFFIX);
+#undef SUFFIX
+
+  static const constexpr char* self_name_ = "SimulatorRPCChannel";
+  static const constexpr char* default_cpu_ = "v68";
+  std::string cpu_;
+
+  HEX_VA_t dispatch_v_, message_buffer_v_;
+  std::unique_ptr<HexagonWrapper> sim_;
+
+  // Sim configuration routines.
+  bool Configure(string_list& opts);  // NOLINT(*)
+
+  bool HandleAHBBusPenalty(string_list& rest);      // NOLINT(*)
+  bool HandleAHBBusRatio(string_list& rest);        // NOLINT(*)
+  bool HandleAHBHighAddr(string_list& rest);        // NOLINT(*)
+  bool HandleAHBLowAddr(string_list& rest);         // NOLINT(*)
+  bool HandleAXI2BusPenalty(string_list& rest);     // NOLINT(*)
+  bool HandleAXI2BusRatio(string_list& rest);       // NOLINT(*)
+  bool HandleAXI2HighAddr(string_list& rest);       // NOLINT(*)
+  bool HandleAXI2LowAddr(string_list& rest);        // NOLINT(*)
+  bool HandleBuildTag(string_list& rest);           // NOLINT(*)
+  bool HandleBusPenalty(string_list& rest);         // NOLINT(*)
+  bool HandleBusRatio(string_list& rest);           // NOLINT(*)
+  bool HandleBusTrace(string_list& rest);           // NOLINT(*)
+  bool HandleBypassIdle(string_list& rest);         // NOLINT(*)
+  bool HandleConnectionTimeout(string_list& rest);  // NOLINT(*)
+  bool HandleCoprocTrace(string_list& rest);        // NOLINT(*)
+  bool HandleCoreDump(string_list& rest);           // NOLINT(*)
+  bool HandleCosimFile(string_list& rest);          // NOLINT(*)
+  bool HandleDCacheTrace(string_list& rest);        // NOLINT(*)
+  bool HandleDSPClock(string_list& rest);           // NOLINT(*)
+  bool HandleETMCFGBase(string_list& rest);         // NOLINT(*)
+  bool HandleGDBServ(string_list& rest);            // NOLINT(*)
+  bool HandleHVXLength(string_list& rest);          // NOLINT(*)
+  bool HandleICacheTrace(string_list& rest);        // NOLINT(*)
+  bool HandleL2CacheTrace(string_list& rest);       // NOLINT(*)
+  bool HandleL2CFGBase(string_list& rest);          // NOLINT(*)
+  bool HandleL2TCMBase(string_list& rest);          // NOLINT(*)
+  bool HandleMemFillRand(string_list& rest);        // NOLINT(*)
+  bool HandleMemFill(string_list& rest);            // NOLINT(*)
+  bool HandleMemTrace(string_list& rest);           // NOLINT(*)
+  bool HandleNullPtr(string_list& rest);            // NOLINT(*)
+  bool HandlePacketAnalyze(string_list& rest);      // NOLINT(*)
+  bool HandlePCFilter(string_list& rest);           // NOLINT(*)
+  bool HandlePCTraceMin(string_list& rest);         // NOLINT(*)
+  bool HandlePCTraceNano(string_list& rest);        // NOLINT(*)
+  bool HandlePCTrace(string_list& rest);            // NOLINT(*)
+  bool HandlePMUStatsFile(string_list& rest);       // NOLINT(*)
+  bool HandleProfile(string_list& rest);            // NOLINT(*)
+  bool HandleProfileTimeZero(string_list& rest);    // NOLINT(*)
+  bool HandleQuiet(string_list& rest);              // NOLINT(*)
+  bool HandleReconnect(string_list& rest);          // NOLINT(*)
+  bool HandleRTOS(string_list& rest);               // NOLINT(*)
+  bool HandleSimErr(string_list& rest);             // NOLINT(*)
+  bool HandleSimIn(string_list& rest);              // NOLINT(*)
+  bool HandleSimOut(string_list& rest);             // NOLINT(*)
+  bool HandleStackStart(string_list& rest);         // NOLINT(*)
+  bool HandleStallTrace(string_list& rest);         // NOLINT(*)
+  bool HandleStatsFile(string_list& rest);          // NOLINT(*)
+  bool HandleSubsystemBase(string_list& rest);      // NOLINT(*)
+  bool HandleSymFile(string_list& rest);            // NOLINT(*)
+  bool HandleTCM(string_list& rest);                // NOLINT(*)
+  bool HandleTCMHighAddr(string_list& rest);        // NOLINT(*)
+  bool HandleTCMLowAddr(string_list& rest);         // NOLINT(*)
+  bool HandleTimeFilterNS(string_list& rest);       // NOLINT(*)
+  bool HandleTiming(string_list& rest);             // NOLINT(*)
+  bool HandleUArchTrace(string_list& rest);         // NOLINT(*)
+  bool HandleUseFS(string_list& rest);              // NOLINT(*)
+  bool HandleV2PTranslation(string_list& rest);     // NOLINT(*)
+  bool HandleVerbose(string_list& rest);            // NOLINT(*)
+
+  using MaybeUInt64 = detail::Optional<uint64_t>;
+  using MaybeUIntRange = std::pair<MaybeUInt64, MaybeUInt64>;
+
+  bool should_parse_next(const string_list& rest);
+  detail::Optional<HEXAPI_Interval> to_interval(const detail::MaybeString& str);
+  detail::Optional<HEXAPI_TimingMode> to_timingmode(const detail::MaybeString& str);
+  detail::Optional<HEXAPI_VerboseMode> to_verbosemode(const detail::MaybeString& str);
+  detail::Optional<HEXAPI_Nullptr> to_nullptr(const detail::MaybeString& str);
+
+  MaybeUIntRange ahb_, axi2_;
+  detail::Optional<uint32_t> debug_port_;
+
+  using OptionHandler = bool (SimulatorRPCChannel::*)(string_list&);
+  static std::map<std::string, OptionHandler> opt_map_;
+};
+
+const constexpr char* SimulatorRPCChannel::self_name_;
+const constexpr char* SimulatorRPCChannel::default_cpu_;
+
+decltype(SimulatorRPCChannel::opt_map_) SimulatorRPCChannel::opt_map_ = {
+    {"--ahbbuspenalty", &SimulatorRPCChannel::HandleAHBBusPenalty},
+    {"--ahbbusratio", &SimulatorRPCChannel::HandleAHBBusRatio},
+    {"--ahb:highaddr", &SimulatorRPCChannel::HandleAHBHighAddr},
+    {"--ahb:lowaddr", &SimulatorRPCChannel::HandleAHBLowAddr},
+    {"--axi2buspenalty", &SimulatorRPCChannel::HandleAXI2BusPenalty},
+    {"--axi2busratio", &SimulatorRPCChannel::HandleAXI2BusRatio},
+    {"--axi2:highaddr", &SimulatorRPCChannel::HandleAXI2HighAddr},
+    {"--axi2:lowaddr", &SimulatorRPCChannel::HandleAXI2LowAddr},
+    {"-b", &SimulatorRPCChannel::HandleBusTrace},
+    {"--build_tag", &SimulatorRPCChannel::HandleBuildTag},
+    {"--buspenalty", &SimulatorRPCChannel::HandleBusPenalty},
+    {"--busratio", &SimulatorRPCChannel::HandleBusRatio},
+    {"--bustrace", &SimulatorRPCChannel::HandleBusTrace},
+    {"--bypass_idle", &SimulatorRPCChannel::HandleBypassIdle},
+    {"--connection_timeout", &SimulatorRPCChannel::HandleConnectionTimeout},
+    {"--coproctrace", &SimulatorRPCChannel::HandleCoprocTrace},
+    {"--coredump", &SimulatorRPCChannel::HandleCoreDump},
+    {"--cosim_file", &SimulatorRPCChannel::HandleCosimFile},
+    {"--dcachetrace", &SimulatorRPCChannel::HandleDCacheTrace},
+    {"--dsp_clock", &SimulatorRPCChannel::HandleDSPClock},
+    {"-E", &SimulatorRPCChannel::HandleSimErr},
+    {"--etm_base", &SimulatorRPCChannel::HandleETMCFGBase},
+    {"--etmcfg_base", &SimulatorRPCChannel::HandleETMCFGBase},
+    {"--gdbserv", &SimulatorRPCChannel::HandleGDBServ},
+    {"-G", &SimulatorRPCChannel::HandleGDBServ},
+    {"--hvx_length", &SimulatorRPCChannel::HandleHVXLength},
+    {"--icachetrace", &SimulatorRPCChannel::HandleICacheTrace},
+    {"-I", &SimulatorRPCChannel::HandleSimIn},
+    {"--l2cachetrace", &SimulatorRPCChannel::HandleL2CacheTrace},
+    {"--l2cfg_base", &SimulatorRPCChannel::HandleL2CFGBase},
+    {"--l2tcm_base", &SimulatorRPCChannel::HandleL2TCMBase},
+    {"--memfill", &SimulatorRPCChannel::HandleMemFill},
+    {"--memfill_rand", &SimulatorRPCChannel::HandleMemFillRand},
+    {"--memtrace", &SimulatorRPCChannel::HandleMemTrace},
+    {"-m", &SimulatorRPCChannel::HandleMemTrace},
+    {"--nullptr", &SimulatorRPCChannel::HandleNullPtr},
+    {"-O", &SimulatorRPCChannel::HandleSimOut},
+    {"--packet_analyze", &SimulatorRPCChannel::HandlePacketAnalyze},
+    {"--pcfilter", &SimulatorRPCChannel::HandlePCFilter},
+    {"--pctrace", &SimulatorRPCChannel::HandlePCTrace},
+    {"--pctrace_min", &SimulatorRPCChannel::HandlePCTraceMin},
+    {"--pctrace_nano", &SimulatorRPCChannel::HandlePCTraceNano},
+    {"-p", &SimulatorRPCChannel::HandleProfile},
+    {"--pmu_statsfile", &SimulatorRPCChannel::HandlePMUStatsFile},
+    {"--profile", &SimulatorRPCChannel::HandleProfile},
+    {"--profile_timezero", &SimulatorRPCChannel::HandleProfileTimeZero},
+    {"-q", &SimulatorRPCChannel::HandleQuiet},
+    {"--quiet", &SimulatorRPCChannel::HandleQuiet},
+    {"--reconnect", &SimulatorRPCChannel::HandleReconnect},
+    {"--rtos", &SimulatorRPCChannel::HandleRTOS},
+    {"-S", &SimulatorRPCChannel::HandleStatsFile},
+    {"--sim_err", &SimulatorRPCChannel::HandleSimErr},
+    {"--sim_in", &SimulatorRPCChannel::HandleSimIn},
+    {"--sim_out", &SimulatorRPCChannel::HandleSimOut},
+    {"--stackstart", &SimulatorRPCChannel::HandleStackStart},
+    {"--stalltrace", &SimulatorRPCChannel::HandleStallTrace},
+    {"--statsfile", &SimulatorRPCChannel::HandleStatsFile},
+    {"--subsystem_base", &SimulatorRPCChannel::HandleSubsystemBase},
+    {"--symfile", &SimulatorRPCChannel::HandleSymFile},
+    {"--tcm", &SimulatorRPCChannel::HandleTCM},
+    {"--tcm:highaddr", &SimulatorRPCChannel::HandleTCMHighAddr},
+    {"--tcm:lowaddr", &SimulatorRPCChannel::HandleTCMLowAddr},
+    {"-t", &SimulatorRPCChannel::HandlePCTrace},
+    {"--timefilter_ns", &SimulatorRPCChannel::HandleTimeFilterNS},
+    {"--timing", &SimulatorRPCChannel::HandleTiming},
+    {"--uarchtrace", &SimulatorRPCChannel::HandleUArchTrace},
+    {"-u", &SimulatorRPCChannel::HandlePCTraceMin},
+    {"--usefs", &SimulatorRPCChannel::HandleUseFS},
+    {"--v2p_translation", &SimulatorRPCChannel::HandleV2PTranslation},
+    {"--verbose", &SimulatorRPCChannel::HandleVerbose},
+};
+
+std::string SimulatorRPCChannel::Status_::str() const {
+  switch (s) {
+    case HEX_STAT_ERROR:
+      return "HEX_STAT_ERROR";
+    case HEX_STAT_SUCCESS:
+      return "HEX_STAT_SUCCESS";
+    case HEX_STAT_CANNOT_CONFIG:
+      return "HEX_STAT_CANNOT_CONFIG";
+    case HEX_STAT_INVALID_ARGS:
+      return "HEX_STAT_INVALID_ARGS";
+    case HEX_STAT_RANGE_ERROR:
+      return "HEX_STAT_RANGE_ERROR";
+    case HEX_STAT_FILE_ACCESS_ERROR:
+      return "HEX_STAT_FILE_ACCESS_ERROR";
+    case HEX_STAT_DEVICE_NOT_FOUND:
+      return "HEX_STAT_DEVICE_NOT_FOUND";
+    case HEX_STAT_MEM_ACCESS_ERROR:
+      return "HEX_STAT_MEM_ACCESS_ERROR";
+    case HEX_STAT_CANNOT_TRANSLATE:
+      return "HEX_STAT_CANNOT_TRANSLATE";
+    case HEX_STAT_NO_ACTIVE_THREADS:
+      return "HEX_STAT_NO_ACTIVE_THREADS";
+    case HEX_STAT_LOAD_ELF_ERROR:
+      return "HEX_STAT_LOAD_ELF_ERROR";
+    case HEX_STAT_CORE_RESET:
+      return "HEX_STAT_CORE_RESET";
+    default:
+      break;
+  }
+  return std::to_string(static_cast<int>(s));
+}
+
+std::string SimulatorRPCChannel::Core_::str() const {
+  switch (s) {
+    case HEX_CORE_SUCCESS:
+      return "HEX_CORE_SUCCESS";
+    case HEX_CORE_FINISHED:
+      return "HEX_CORE_FINISHED";
+    case HEX_CORE_RESET:
+      return "HEX_CORE_RESET";
+    case HEX_CORE_BREAKPOINT:
+      return "HEX_CORE_BREAKPOINT";
+    case HEX_CORE_ASYNCHRONOUS_BREAK:
+      return "HEX_CORE_ASYNCHRONOUS_BREAK";
+    case HEX_CORE_ERROR:
+      return "HEX_CORE_ERROR";
+    default:
+      break;
+  }
+  return std::to_string(static_cast<int>(s));
+}
+
+std::string SimulatorRPCChannel::Cpu_::str() const {
+  switch (c) {
+    case HEX_CPU_V65:
+      return "v65";
+    case HEX_CPU_V66:
+      return "v66";
+    case HEX_CPU_V68:
+      return "v68";
+    case HEX_CPU_V69:
+      return "v69";
+    default:
+      break;
+  }
+  return default_cpu_;
+}
+
+SimulatorRPCChannel::SDKInfo_::SDKInfo_(const std::string& sdk_root, const std::string& cpu)
+    : root(sdk_root) {
+  qurt_root = root + "/rtos/qurt/compute" + cpu;
+  runelf = qurt_root + "/sdksim_bin/runelf.pbn";
+
+  // The "run_main_on_hexagon_sim" binary lives in a subdirectory that looks
+  // like "[...]on_hexagon/ship/hexagon_toolv84_v68/run_main_on_hexagon_sim".
+  // We need to get the right "hexagon_toolv..." component, based mostly on
+  // the cpu version.
+  std::vector<std::string> dir_names;
+
+  DIR* dir = opendir((root + "/libs/run_main_on_hexagon/ship").c_str());
+  ICHECK(dir != nullptr) << "Cannot read directory " << root + "/libs/run_main_on_hexagon/ship";
+  while (dirent* d = readdir(dir)) {
+    if (d->d_type != DT_DIR) continue;
+
+    std::string name = d->d_name;
+    // Note: The first substr is always safe, and the second only executes
+    // when "name" is at least 13 characters long.
+    if (name.substr(0, 13) == "hexagon_toolv" && name.substr(name.size() - 3, 3) == cpu) {
+      dir_names.push_back(name);
+    }
+  }
+  closedir(dir);
+  ICHECK(!dir_names.empty());
+
+  auto max_it = std::max_element(dir_names.begin(), dir_names.end());
+  runmain = root + "/libs/run_main_on_hexagon/ship/" + *max_it + "/run_main_on_hexagon_sim";
+}
+
+HEX_8u_t SimulatorRPCChannel::PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV,
+                                                   HEX_8u_t RttV, HEX_8u_t RxxV, HEX_1u_t imm) {
+  // Rssv = combine(&message_buffer, &dispatch)
+  auto* rpc = reinterpret_cast<SimulatorRPCChannel*>(handle);
+  rpc->dispatch_v_ = RssV & ~0u;  // ~0u is uint32_t
+  rpc->message_buffer_v_ = RssV >> 32;
+
+  LOG(INFO) << "dispatch:" << reinterpret_cast<void*>(rpc->dispatch_v_)

Review comment:
       `DLOG` or `VLOG` (can be done in a follow up)




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

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

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



[GitHub] [tvm] kparzysz-quic commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

Posted by GitBox <gi...@apache.org>.
kparzysz-quic commented on a change in pull request #10361:
URL: https://github.com/apache/tvm/pull/10361#discussion_r813014494



##########
File path: apps/hexagon_api/CMakeLists.txt
##########
@@ -23,6 +23,31 @@ else()
 endif()
 file(MAKE_DIRECTORY ${HEXAGON_API_BINARY_DIR})
 
+# Build X86 binaries:
+# - tvm_rpc_x86
+
+ExternalProject_Add(x86_tvm_runtime_rpc

Review comment:
       Build a cpp RPC server app for x86 with the Hexagon RPC code enabled.




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

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

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



[GitHub] [tvm] mehrdadh commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

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



##########
File path: src/runtime/hexagon/rpc/simulator/session.cc
##########
@@ -0,0 +1,1322 @@
+/*
+ * 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.
+ */
+
+#include <HexagonWrapper.h>
+#include <dmlc/optional.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+// POSIX includes
+#include <dirent.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <deque>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <utility>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "hexagon_sim_proto.h"
+
+#define CHECKED_CALL(func, ...)                                               \
+  do {                                                                        \
+    HEXAPI_Status s = sim_->func(__VA_ARGS__);                                \
+    ICHECK_EQ(s, HEX_STAT_SUCCESS)                                            \
+        << self_name_ << ": " #func " failed with code " << Status_{s}.str(); \
+  } while (false)
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+using string_list = std::deque<std::string>;
+
+namespace detail {
+
+// An "Optional" class, originally a replacement for llvm::Optional, then an
+// extension of dmlc::optional to make it compatible with C++17's std::optional.
+template <typename T>
+struct Optional : public dmlc::optional<T> {
+  using dmlc::optional<T>::optional;
+  using dmlc::optional<T>::operator=;
+  Optional(const T& val) : dmlc::optional<T>(val) {}  // NOLINT(*)
+
+  T* operator->() { return &this->operator*(); }
+  const T* operator->() const { return &this->operator*(); }
+};
+
+// Replacement for llvm::StringSwitch.
+template <typename T>
+class StringSwitch {
+ public:
+  explicit StringSwitch(const std::string& key) : key(key) {}
+  operator T() const {
+    auto f = map.find(key);
+    if (f != map.end()) {
+      return f->second;
+    }
+    ICHECK(static_cast<bool>(def_val)) << "default value not set";
+    return *def_val;
+  }
+  StringSwitch& Case(const std::string& key, T val) {
+    map.insert(std::make_pair(key, val));
+    return *this;
+  }
+  StringSwitch& Default(T val) {
+    ICHECK(!static_cast<bool>(def_val)) << "default value already set";
+    def_val = val;
+    return *this;
+  }
+
+ private:
+  const std::string key;
+  std::map<std::string, T> map;
+  Optional<T> def_val;
+};
+
+using MaybeString = Optional<std::string>;
+
+MaybeString front(const string_list& deq) {
+  return !deq.empty() ? MaybeString(deq.front()) : MaybeString();
+}
+
+MaybeString pop_front(string_list& deq) {  // NOLINT(*)
+  if (deq.empty()) return MaybeString();
+  std::string f = deq.front();
+  deq.pop_front();
+  return MaybeString(f);
+}
+
+// Functions used when parsing the argument string.
+
+Optional<int64_t> to_int(const MaybeString& str) {
+  auto none = Optional<int64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      int64_t val = std::stoll(*str, &pos, 0);
+      return pos == str->size() ? Optional<int64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<uint64_t> to_uint(const MaybeString& str) {
+  auto none = Optional<uint64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      uint64_t val = std::stoull(*str, &pos, 0);
+      return pos == str->size() ? Optional<uint64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<float> to_float(const MaybeString& str) {
+  auto none = Optional<float>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      float val = std::stof(*str, &pos);
+      return pos == str->size() ? Optional<float>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<bool> to_bool(const MaybeString& str) {
+  auto none = Optional<bool>();
+  if (auto num = to_int(str)) {
+    if (*num == 0) return false;
+    if (*num == 1) return true;
+    return none;
+  }
+  if (str) {
+    if (*str == "true" || *str == "TRUE") return true;
+    if (*str == "false" || *str == "FALSE") return false;
+  }
+  return none;
+}
+
+template <typename T>
+using MaybeRange = Optional<std::pair<T, T>>;
+
+template <typename T, Optional<T> Parse(const MaybeString&)>
+MaybeRange<T> to_range(const MaybeString& str) {
+  auto none = MaybeRange<T>();
+  if (str && !str->empty()) {
+    auto n = str->find('-', 1);
+    if (n != std::string::npos) {
+      auto begin = Parse(str->substr(0, n));
+      auto end = Parse(str->substr(n + 1, str->size() - n - 1));
+      if (begin && end) {
+        return std::make_pair(*begin, *end);
+      }
+    }
+  }
+  return none;
+}
+
+}  // namespace detail
+
+class SimulatorRPCChannel final : public RPCChannel {
+ public:
+  SimulatorRPCChannel(std::string args);
+  ~SimulatorRPCChannel() final;
+  size_t Send(const void* data, size_t size) final;
+  size_t Recv(void* data, size_t size) final;
+
+ private:
+  struct Status_ {
+    HEXAPI_Status s;
+    std::string str() const;
+  };
+  struct Core_ {
+    HEXAPI_CoreState s;
+    std::string str() const;
+  };
+  struct Cpu_ {
+    HEXAPI_Cpu c;
+    std::string str() const;
+  };
+  struct SDKInfo_ {
+    SDKInfo_(const std::string& sdk_root, const std::string& cpu);
+    std::string root;
+    std::string qurt_root;  // sdk_root/rtos/qurt/computevNN.
+    std::string runelf;     // Path to runelf.pbn.
+    std::string runmain;    // Path to run_main_on_hexagon.
+  };
+
+  Message SendMsg(Message msg);
+  Message SendMsg(uint32_t code, uint32_t len, uint32_t va);
+  void ReadFromProcess(void* host_dst, HEX_VA_t src, size_t len);
+  void WriteToProcess(HEX_VA_t dst, const void* host_src, size_t len);
+
+  static HEX_8u_t PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV, HEX_8u_t RttV,
+                                       HEX_8u_t RxxV, HEX_1u_t imm);
+
+  detail::Optional<HEXAPI_Cpu> GetCPU(const detail::MaybeString& cpu_str);
+
+  // File name templates for mkstemps.
+#define SUFFIX ".cfg"
+  static constexpr int template_length_ = strlen("temp-xxxx-XXXXXX" SUFFIX) + 1;
+  char osam_file_[template_length_] = "temp-osam-XXXXXX" SUFFIX;
+  char cosim_file_[template_length_] = "temp-q6ss-XXXXXX" SUFFIX;
+  const int suffix_len_ = strlen(SUFFIX);
+#undef SUFFIX
+
+  static const constexpr char* self_name_ = "SimulatorRPCChannel";
+  static const constexpr char* default_cpu_ = "v68";
+  std::string cpu_;
+
+  HEX_VA_t dispatch_v_, message_buffer_v_;
+  std::unique_ptr<HexagonWrapper> sim_;
+
+  // Sim configuration routines.

Review comment:
       can you move all of these configurations to a header file and reuse it?




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

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

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



[GitHub] [tvm] kparzysz-quic commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

Posted by GitBox <gi...@apache.org>.
kparzysz-quic commented on a change in pull request #10361:
URL: https://github.com/apache/tvm/pull/10361#discussion_r813016527



##########
File path: src/runtime/hexagon/rpc/simulator/session.cc
##########
@@ -0,0 +1,1322 @@
+/*
+ * 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.
+ */
+
+#include <HexagonWrapper.h>
+#include <dmlc/optional.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+// POSIX includes
+#include <dirent.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <deque>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <utility>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "hexagon_sim_proto.h"
+
+#define CHECKED_CALL(func, ...)                                               \
+  do {                                                                        \
+    HEXAPI_Status s = sim_->func(__VA_ARGS__);                                \
+    ICHECK_EQ(s, HEX_STAT_SUCCESS)                                            \
+        << self_name_ << ": " #func " failed with code " << Status_{s}.str(); \
+  } while (false)
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+using string_list = std::deque<std::string>;
+
+namespace detail {
+
+// An "Optional" class, originally a replacement for llvm::Optional, then an
+// extension of dmlc::optional to make it compatible with C++17's std::optional.
+template <typename T>
+struct Optional : public dmlc::optional<T> {
+  using dmlc::optional<T>::optional;
+  using dmlc::optional<T>::operator=;
+  Optional(const T& val) : dmlc::optional<T>(val) {}  // NOLINT(*)
+
+  T* operator->() { return &this->operator*(); }
+  const T* operator->() const { return &this->operator*(); }
+};
+
+// Replacement for llvm::StringSwitch.
+template <typename T>
+class StringSwitch {
+ public:
+  explicit StringSwitch(const std::string& key) : key(key) {}
+  operator T() const {
+    auto f = map.find(key);
+    if (f != map.end()) {
+      return f->second;
+    }
+    ICHECK(static_cast<bool>(def_val)) << "default value not set";
+    return *def_val;
+  }
+  StringSwitch& Case(const std::string& key, T val) {
+    map.insert(std::make_pair(key, val));
+    return *this;
+  }
+  StringSwitch& Default(T val) {
+    ICHECK(!static_cast<bool>(def_val)) << "default value already set";
+    def_val = val;
+    return *this;
+  }
+
+ private:
+  const std::string key;
+  std::map<std::string, T> map;
+  Optional<T> def_val;
+};
+
+using MaybeString = Optional<std::string>;
+
+MaybeString front(const string_list& deq) {
+  return !deq.empty() ? MaybeString(deq.front()) : MaybeString();
+}
+
+MaybeString pop_front(string_list& deq) {  // NOLINT(*)
+  if (deq.empty()) return MaybeString();
+  std::string f = deq.front();
+  deq.pop_front();
+  return MaybeString(f);
+}
+
+// Functions used when parsing the argument string.
+
+Optional<int64_t> to_int(const MaybeString& str) {
+  auto none = Optional<int64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      int64_t val = std::stoll(*str, &pos, 0);
+      return pos == str->size() ? Optional<int64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<uint64_t> to_uint(const MaybeString& str) {
+  auto none = Optional<uint64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      uint64_t val = std::stoull(*str, &pos, 0);
+      return pos == str->size() ? Optional<uint64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<float> to_float(const MaybeString& str) {
+  auto none = Optional<float>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      float val = std::stof(*str, &pos);
+      return pos == str->size() ? Optional<float>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<bool> to_bool(const MaybeString& str) {
+  auto none = Optional<bool>();
+  if (auto num = to_int(str)) {
+    if (*num == 0) return false;
+    if (*num == 1) return true;
+    return none;
+  }
+  if (str) {
+    if (*str == "true" || *str == "TRUE") return true;
+    if (*str == "false" || *str == "FALSE") return false;
+  }
+  return none;
+}
+
+template <typename T>
+using MaybeRange = Optional<std::pair<T, T>>;
+
+template <typename T, Optional<T> Parse(const MaybeString&)>
+MaybeRange<T> to_range(const MaybeString& str) {
+  auto none = MaybeRange<T>();
+  if (str && !str->empty()) {
+    auto n = str->find('-', 1);
+    if (n != std::string::npos) {
+      auto begin = Parse(str->substr(0, n));
+      auto end = Parse(str->substr(n + 1, str->size() - n - 1));
+      if (begin && end) {
+        return std::make_pair(*begin, *end);
+      }
+    }
+  }
+  return none;
+}
+
+}  // namespace detail
+
+class SimulatorRPCChannel final : public RPCChannel {
+ public:
+  SimulatorRPCChannel(std::string args);
+  ~SimulatorRPCChannel() final;
+  size_t Send(const void* data, size_t size) final;
+  size_t Recv(void* data, size_t size) final;
+
+ private:
+  struct Status_ {
+    HEXAPI_Status s;
+    std::string str() const;
+  };
+  struct Core_ {
+    HEXAPI_CoreState s;
+    std::string str() const;
+  };
+  struct Cpu_ {
+    HEXAPI_Cpu c;
+    std::string str() const;
+  };
+  struct SDKInfo_ {
+    SDKInfo_(const std::string& sdk_root, const std::string& cpu);
+    std::string root;
+    std::string qurt_root;  // sdk_root/rtos/qurt/computevNN.
+    std::string runelf;     // Path to runelf.pbn.
+    std::string runmain;    // Path to run_main_on_hexagon.
+  };
+
+  Message SendMsg(Message msg);
+  Message SendMsg(uint32_t code, uint32_t len, uint32_t va);
+  void ReadFromProcess(void* host_dst, HEX_VA_t src, size_t len);
+  void WriteToProcess(HEX_VA_t dst, const void* host_src, size_t len);
+
+  static HEX_8u_t PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV, HEX_8u_t RttV,
+                                       HEX_8u_t RxxV, HEX_1u_t imm);
+
+  detail::Optional<HEXAPI_Cpu> GetCPU(const detail::MaybeString& cpu_str);
+
+  // File name templates for mkstemps.
+#define SUFFIX ".cfg"
+  static constexpr int template_length_ = strlen("temp-xxxx-XXXXXX" SUFFIX) + 1;
+  char osam_file_[template_length_] = "temp-osam-XXXXXX" SUFFIX;
+  char cosim_file_[template_length_] = "temp-q6ss-XXXXXX" SUFFIX;
+  const int suffix_len_ = strlen(SUFFIX);
+#undef SUFFIX
+
+  static const constexpr char* self_name_ = "SimulatorRPCChannel";
+  static const constexpr char* default_cpu_ = "v68";
+  std::string cpu_;
+
+  HEX_VA_t dispatch_v_, message_buffer_v_;
+  std::unique_ptr<HexagonWrapper> sim_;
+
+  // Sim configuration routines.

Review comment:
       All of this configuration code was taken from `src/runtime/hexagon/android/sim/hexagon_device_sim.cpp`.




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

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

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



[GitHub] [tvm] kparzysz-quic commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

Posted by GitBox <gi...@apache.org>.
kparzysz-quic commented on a change in pull request #10361:
URL: https://github.com/apache/tvm/pull/10361#discussion_r813321105



##########
File path: src/runtime/hexagon/rpc/simulator/session.cc
##########
@@ -0,0 +1,1322 @@
+/*
+ * 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.
+ */
+
+#include <HexagonWrapper.h>
+#include <dmlc/optional.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+// POSIX includes
+#include <dirent.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <deque>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <utility>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "hexagon_sim_proto.h"
+
+#define CHECKED_CALL(func, ...)                                               \
+  do {                                                                        \
+    HEXAPI_Status s = sim_->func(__VA_ARGS__);                                \
+    ICHECK_EQ(s, HEX_STAT_SUCCESS)                                            \
+        << self_name_ << ": " #func " failed with code " << Status_{s}.str(); \
+  } while (false)
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+using string_list = std::deque<std::string>;
+
+namespace detail {
+
+// An "Optional" class, originally a replacement for llvm::Optional, then an
+// extension of dmlc::optional to make it compatible with C++17's std::optional.
+template <typename T>
+struct Optional : public dmlc::optional<T> {
+  using dmlc::optional<T>::optional;
+  using dmlc::optional<T>::operator=;
+  Optional(const T& val) : dmlc::optional<T>(val) {}  // NOLINT(*)
+
+  T* operator->() { return &this->operator*(); }
+  const T* operator->() const { return &this->operator*(); }
+};
+
+// Replacement for llvm::StringSwitch.
+template <typename T>
+class StringSwitch {
+ public:
+  explicit StringSwitch(const std::string& key) : key(key) {}
+  operator T() const {
+    auto f = map.find(key);
+    if (f != map.end()) {
+      return f->second;
+    }
+    ICHECK(static_cast<bool>(def_val)) << "default value not set";
+    return *def_val;
+  }
+  StringSwitch& Case(const std::string& key, T val) {
+    map.insert(std::make_pair(key, val));
+    return *this;
+  }
+  StringSwitch& Default(T val) {
+    ICHECK(!static_cast<bool>(def_val)) << "default value already set";
+    def_val = val;
+    return *this;
+  }
+
+ private:
+  const std::string key;
+  std::map<std::string, T> map;
+  Optional<T> def_val;
+};
+
+using MaybeString = Optional<std::string>;
+
+MaybeString front(const string_list& deq) {
+  return !deq.empty() ? MaybeString(deq.front()) : MaybeString();
+}
+
+MaybeString pop_front(string_list& deq) {  // NOLINT(*)
+  if (deq.empty()) return MaybeString();
+  std::string f = deq.front();
+  deq.pop_front();
+  return MaybeString(f);
+}
+
+// Functions used when parsing the argument string.
+
+Optional<int64_t> to_int(const MaybeString& str) {
+  auto none = Optional<int64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      int64_t val = std::stoll(*str, &pos, 0);
+      return pos == str->size() ? Optional<int64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<uint64_t> to_uint(const MaybeString& str) {
+  auto none = Optional<uint64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      uint64_t val = std::stoull(*str, &pos, 0);
+      return pos == str->size() ? Optional<uint64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<float> to_float(const MaybeString& str) {
+  auto none = Optional<float>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      float val = std::stof(*str, &pos);
+      return pos == str->size() ? Optional<float>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<bool> to_bool(const MaybeString& str) {
+  auto none = Optional<bool>();
+  if (auto num = to_int(str)) {
+    if (*num == 0) return false;
+    if (*num == 1) return true;
+    return none;
+  }
+  if (str) {
+    if (*str == "true" || *str == "TRUE") return true;
+    if (*str == "false" || *str == "FALSE") return false;
+  }
+  return none;
+}
+
+template <typename T>
+using MaybeRange = Optional<std::pair<T, T>>;
+
+template <typename T, Optional<T> Parse(const MaybeString&)>
+MaybeRange<T> to_range(const MaybeString& str) {
+  auto none = MaybeRange<T>();
+  if (str && !str->empty()) {
+    auto n = str->find('-', 1);
+    if (n != std::string::npos) {
+      auto begin = Parse(str->substr(0, n));
+      auto end = Parse(str->substr(n + 1, str->size() - n - 1));
+      if (begin && end) {
+        return std::make_pair(*begin, *end);
+      }
+    }
+  }
+  return none;
+}
+
+}  // namespace detail
+
+class SimulatorRPCChannel final : public RPCChannel {
+ public:
+  SimulatorRPCChannel(std::string args);
+  ~SimulatorRPCChannel() final;
+  size_t Send(const void* data, size_t size) final;
+  size_t Recv(void* data, size_t size) final;
+
+ private:
+  struct Status_ {
+    HEXAPI_Status s;
+    std::string str() const;
+  };
+  struct Core_ {
+    HEXAPI_CoreState s;
+    std::string str() const;
+  };
+  struct Cpu_ {
+    HEXAPI_Cpu c;
+    std::string str() const;
+  };
+  struct SDKInfo_ {
+    SDKInfo_(const std::string& sdk_root, const std::string& cpu);
+    std::string root;
+    std::string qurt_root;  // sdk_root/rtos/qurt/computevNN.
+    std::string runelf;     // Path to runelf.pbn.
+    std::string runmain;    // Path to run_main_on_hexagon.
+  };
+
+  Message SendMsg(Message msg);
+  Message SendMsg(uint32_t code, uint32_t len, uint32_t va);
+  void ReadFromProcess(void* host_dst, HEX_VA_t src, size_t len);
+  void WriteToProcess(HEX_VA_t dst, const void* host_src, size_t len);
+
+  static HEX_8u_t PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV, HEX_8u_t RttV,
+                                       HEX_8u_t RxxV, HEX_1u_t imm);
+
+  detail::Optional<HEXAPI_Cpu> GetCPU(const detail::MaybeString& cpu_str);
+
+  // File name templates for mkstemps.
+#define SUFFIX ".cfg"
+  static constexpr int template_length_ = strlen("temp-xxxx-XXXXXX" SUFFIX) + 1;
+  char osam_file_[template_length_] = "temp-osam-XXXXXX" SUFFIX;
+  char cosim_file_[template_length_] = "temp-q6ss-XXXXXX" SUFFIX;
+  const int suffix_len_ = strlen(SUFFIX);
+#undef SUFFIX
+
+  static const constexpr char* self_name_ = "SimulatorRPCChannel";
+  static const constexpr char* default_cpu_ = "v68";
+  std::string cpu_;
+
+  HEX_VA_t dispatch_v_, message_buffer_v_;
+  std::unique_ptr<HexagonWrapper> sim_;
+
+  // Sim configuration routines.

Review comment:
       The offload code will be removed, so this will be the only place where it appears.  The files in `src/runtime/hexagon/android` will be deleted.




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

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

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



[GitHub] [tvm] masahi commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

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



##########
File path: src/runtime/hexagon/rpc/simulator/session.cc
##########
@@ -0,0 +1,1322 @@
+/*
+ * 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.
+ */
+
+#include <HexagonWrapper.h>
+#include <dmlc/optional.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+// POSIX includes
+#include <dirent.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <deque>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <utility>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "hexagon_sim_proto.h"
+
+#define CHECKED_CALL(func, ...)                                               \
+  do {                                                                        \
+    HEXAPI_Status s = sim_->func(__VA_ARGS__);                                \
+    ICHECK_EQ(s, HEX_STAT_SUCCESS)                                            \
+        << self_name_ << ": " #func " failed with code " << Status_{s}.str(); \
+  } while (false)
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+using string_list = std::deque<std::string>;
+
+namespace detail {
+
+// An "Optional" class, originally a replacement for llvm::Optional, then an
+// extension of dmlc::optional to make it compatible with C++17's std::optional.
+template <typename T>
+struct Optional : public dmlc::optional<T> {
+  using dmlc::optional<T>::optional;
+  using dmlc::optional<T>::operator=;
+  Optional(const T& val) : dmlc::optional<T>(val) {}  // NOLINT(*)
+
+  T* operator->() { return &this->operator*(); }
+  const T* operator->() const { return &this->operator*(); }
+};
+
+// Replacement for llvm::StringSwitch.
+template <typename T>
+class StringSwitch {
+ public:
+  explicit StringSwitch(const std::string& key) : key(key) {}
+  operator T() const {
+    auto f = map.find(key);
+    if (f != map.end()) {
+      return f->second;
+    }
+    ICHECK(static_cast<bool>(def_val)) << "default value not set";
+    return *def_val;
+  }
+  StringSwitch& Case(const std::string& key, T val) {
+    map.insert(std::make_pair(key, val));
+    return *this;
+  }
+  StringSwitch& Default(T val) {
+    ICHECK(!static_cast<bool>(def_val)) << "default value already set";
+    def_val = val;
+    return *this;
+  }
+
+ private:
+  const std::string key;
+  std::map<std::string, T> map;
+  Optional<T> def_val;
+};
+
+using MaybeString = Optional<std::string>;
+
+MaybeString front(const string_list& deq) {
+  return !deq.empty() ? MaybeString(deq.front()) : MaybeString();
+}
+
+MaybeString pop_front(string_list& deq) {  // NOLINT(*)
+  if (deq.empty()) return MaybeString();
+  std::string f = deq.front();
+  deq.pop_front();
+  return MaybeString(f);
+}
+
+// Functions used when parsing the argument string.
+
+Optional<int64_t> to_int(const MaybeString& str) {
+  auto none = Optional<int64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      int64_t val = std::stoll(*str, &pos, 0);
+      return pos == str->size() ? Optional<int64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<uint64_t> to_uint(const MaybeString& str) {
+  auto none = Optional<uint64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      uint64_t val = std::stoull(*str, &pos, 0);
+      return pos == str->size() ? Optional<uint64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<float> to_float(const MaybeString& str) {
+  auto none = Optional<float>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      float val = std::stof(*str, &pos);
+      return pos == str->size() ? Optional<float>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<bool> to_bool(const MaybeString& str) {
+  auto none = Optional<bool>();
+  if (auto num = to_int(str)) {
+    if (*num == 0) return false;
+    if (*num == 1) return true;
+    return none;
+  }
+  if (str) {
+    if (*str == "true" || *str == "TRUE") return true;
+    if (*str == "false" || *str == "FALSE") return false;
+  }
+  return none;
+}
+
+template <typename T>
+using MaybeRange = Optional<std::pair<T, T>>;
+
+template <typename T, Optional<T> Parse(const MaybeString&)>
+MaybeRange<T> to_range(const MaybeString& str) {
+  auto none = MaybeRange<T>();
+  if (str && !str->empty()) {
+    auto n = str->find('-', 1);
+    if (n != std::string::npos) {
+      auto begin = Parse(str->substr(0, n));
+      auto end = Parse(str->substr(n + 1, str->size() - n - 1));
+      if (begin && end) {
+        return std::make_pair(*begin, *end);
+      }
+    }
+  }
+  return none;
+}
+
+}  // namespace detail
+
+class SimulatorRPCChannel final : public RPCChannel {
+ public:
+  SimulatorRPCChannel(std::string args);
+  ~SimulatorRPCChannel() final;
+  size_t Send(const void* data, size_t size) final;
+  size_t Recv(void* data, size_t size) final;
+
+ private:
+  struct Status_ {
+    HEXAPI_Status s;
+    std::string str() const;
+  };
+  struct Core_ {
+    HEXAPI_CoreState s;
+    std::string str() const;
+  };
+  struct Cpu_ {
+    HEXAPI_Cpu c;
+    std::string str() const;
+  };
+  struct SDKInfo_ {
+    SDKInfo_(const std::string& sdk_root, const std::string& cpu);
+    std::string root;
+    std::string qurt_root;  // sdk_root/rtos/qurt/computevNN.
+    std::string runelf;     // Path to runelf.pbn.
+    std::string runmain;    // Path to run_main_on_hexagon.
+  };
+
+  Message SendMsg(Message msg);
+  Message SendMsg(uint32_t code, uint32_t len, uint32_t va);
+  void ReadFromProcess(void* host_dst, HEX_VA_t src, size_t len);
+  void WriteToProcess(HEX_VA_t dst, const void* host_src, size_t len);
+
+  static HEX_8u_t PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV, HEX_8u_t RttV,
+                                       HEX_8u_t RxxV, HEX_1u_t imm);
+
+  detail::Optional<HEXAPI_Cpu> GetCPU(const detail::MaybeString& cpu_str);
+
+  // File name templates for mkstemps.
+#define SUFFIX ".cfg"
+  static constexpr int template_length_ = strlen("temp-xxxx-XXXXXX" SUFFIX) + 1;
+  char osam_file_[template_length_] = "temp-osam-XXXXXX" SUFFIX;
+  char cosim_file_[template_length_] = "temp-q6ss-XXXXXX" SUFFIX;
+  const int suffix_len_ = strlen(SUFFIX);
+#undef SUFFIX
+
+  static const constexpr char* self_name_ = "SimulatorRPCChannel";
+  static const constexpr char* default_cpu_ = "v68";
+  std::string cpu_;
+
+  HEX_VA_t dispatch_v_, message_buffer_v_;
+  std::unique_ptr<HexagonWrapper> sim_;
+
+  // Sim configuration routines.
+  bool Configure(string_list& opts);  // NOLINT(*)
+
+  bool HandleAHBBusPenalty(string_list& rest);      // NOLINT(*)
+  bool HandleAHBBusRatio(string_list& rest);        // NOLINT(*)
+  bool HandleAHBHighAddr(string_list& rest);        // NOLINT(*)
+  bool HandleAHBLowAddr(string_list& rest);         // NOLINT(*)
+  bool HandleAXI2BusPenalty(string_list& rest);     // NOLINT(*)
+  bool HandleAXI2BusRatio(string_list& rest);       // NOLINT(*)
+  bool HandleAXI2HighAddr(string_list& rest);       // NOLINT(*)
+  bool HandleAXI2LowAddr(string_list& rest);        // NOLINT(*)
+  bool HandleBuildTag(string_list& rest);           // NOLINT(*)
+  bool HandleBusPenalty(string_list& rest);         // NOLINT(*)
+  bool HandleBusRatio(string_list& rest);           // NOLINT(*)
+  bool HandleBusTrace(string_list& rest);           // NOLINT(*)
+  bool HandleBypassIdle(string_list& rest);         // NOLINT(*)
+  bool HandleConnectionTimeout(string_list& rest);  // NOLINT(*)
+  bool HandleCoprocTrace(string_list& rest);        // NOLINT(*)
+  bool HandleCoreDump(string_list& rest);           // NOLINT(*)
+  bool HandleCosimFile(string_list& rest);          // NOLINT(*)
+  bool HandleDCacheTrace(string_list& rest);        // NOLINT(*)
+  bool HandleDSPClock(string_list& rest);           // NOLINT(*)
+  bool HandleETMCFGBase(string_list& rest);         // NOLINT(*)
+  bool HandleGDBServ(string_list& rest);            // NOLINT(*)
+  bool HandleHVXLength(string_list& rest);          // NOLINT(*)
+  bool HandleICacheTrace(string_list& rest);        // NOLINT(*)
+  bool HandleL2CacheTrace(string_list& rest);       // NOLINT(*)
+  bool HandleL2CFGBase(string_list& rest);          // NOLINT(*)
+  bool HandleL2TCMBase(string_list& rest);          // NOLINT(*)
+  bool HandleMemFillRand(string_list& rest);        // NOLINT(*)
+  bool HandleMemFill(string_list& rest);            // NOLINT(*)
+  bool HandleMemTrace(string_list& rest);           // NOLINT(*)
+  bool HandleNullPtr(string_list& rest);            // NOLINT(*)
+  bool HandlePacketAnalyze(string_list& rest);      // NOLINT(*)
+  bool HandlePCFilter(string_list& rest);           // NOLINT(*)
+  bool HandlePCTraceMin(string_list& rest);         // NOLINT(*)
+  bool HandlePCTraceNano(string_list& rest);        // NOLINT(*)
+  bool HandlePCTrace(string_list& rest);            // NOLINT(*)
+  bool HandlePMUStatsFile(string_list& rest);       // NOLINT(*)
+  bool HandleProfile(string_list& rest);            // NOLINT(*)
+  bool HandleProfileTimeZero(string_list& rest);    // NOLINT(*)
+  bool HandleQuiet(string_list& rest);              // NOLINT(*)
+  bool HandleReconnect(string_list& rest);          // NOLINT(*)
+  bool HandleRTOS(string_list& rest);               // NOLINT(*)
+  bool HandleSimErr(string_list& rest);             // NOLINT(*)
+  bool HandleSimIn(string_list& rest);              // NOLINT(*)
+  bool HandleSimOut(string_list& rest);             // NOLINT(*)
+  bool HandleStackStart(string_list& rest);         // NOLINT(*)
+  bool HandleStallTrace(string_list& rest);         // NOLINT(*)
+  bool HandleStatsFile(string_list& rest);          // NOLINT(*)
+  bool HandleSubsystemBase(string_list& rest);      // NOLINT(*)
+  bool HandleSymFile(string_list& rest);            // NOLINT(*)
+  bool HandleTCM(string_list& rest);                // NOLINT(*)
+  bool HandleTCMHighAddr(string_list& rest);        // NOLINT(*)
+  bool HandleTCMLowAddr(string_list& rest);         // NOLINT(*)
+  bool HandleTimeFilterNS(string_list& rest);       // NOLINT(*)
+  bool HandleTiming(string_list& rest);             // NOLINT(*)
+  bool HandleUArchTrace(string_list& rest);         // NOLINT(*)
+  bool HandleUseFS(string_list& rest);              // NOLINT(*)
+  bool HandleV2PTranslation(string_list& rest);     // NOLINT(*)
+  bool HandleVerbose(string_list& rest);            // NOLINT(*)
+
+  using MaybeUInt64 = detail::Optional<uint64_t>;
+  using MaybeUIntRange = std::pair<MaybeUInt64, MaybeUInt64>;
+
+  bool should_parse_next(const string_list& rest);
+  detail::Optional<HEXAPI_Interval> to_interval(const detail::MaybeString& str);
+  detail::Optional<HEXAPI_TimingMode> to_timingmode(const detail::MaybeString& str);
+  detail::Optional<HEXAPI_VerboseMode> to_verbosemode(const detail::MaybeString& str);
+  detail::Optional<HEXAPI_Nullptr> to_nullptr(const detail::MaybeString& str);
+
+  MaybeUIntRange ahb_, axi2_;
+  detail::Optional<uint32_t> debug_port_;
+
+  using OptionHandler = bool (SimulatorRPCChannel::*)(string_list&);
+  static std::map<std::string, OptionHandler> opt_map_;
+};
+
+const constexpr char* SimulatorRPCChannel::self_name_;
+const constexpr char* SimulatorRPCChannel::default_cpu_;
+
+decltype(SimulatorRPCChannel::opt_map_) SimulatorRPCChannel::opt_map_ = {
+    {"--ahbbuspenalty", &SimulatorRPCChannel::HandleAHBBusPenalty},
+    {"--ahbbusratio", &SimulatorRPCChannel::HandleAHBBusRatio},
+    {"--ahb:highaddr", &SimulatorRPCChannel::HandleAHBHighAddr},
+    {"--ahb:lowaddr", &SimulatorRPCChannel::HandleAHBLowAddr},
+    {"--axi2buspenalty", &SimulatorRPCChannel::HandleAXI2BusPenalty},
+    {"--axi2busratio", &SimulatorRPCChannel::HandleAXI2BusRatio},
+    {"--axi2:highaddr", &SimulatorRPCChannel::HandleAXI2HighAddr},
+    {"--axi2:lowaddr", &SimulatorRPCChannel::HandleAXI2LowAddr},
+    {"-b", &SimulatorRPCChannel::HandleBusTrace},
+    {"--build_tag", &SimulatorRPCChannel::HandleBuildTag},
+    {"--buspenalty", &SimulatorRPCChannel::HandleBusPenalty},
+    {"--busratio", &SimulatorRPCChannel::HandleBusRatio},
+    {"--bustrace", &SimulatorRPCChannel::HandleBusTrace},
+    {"--bypass_idle", &SimulatorRPCChannel::HandleBypassIdle},
+    {"--connection_timeout", &SimulatorRPCChannel::HandleConnectionTimeout},
+    {"--coproctrace", &SimulatorRPCChannel::HandleCoprocTrace},
+    {"--coredump", &SimulatorRPCChannel::HandleCoreDump},
+    {"--cosim_file", &SimulatorRPCChannel::HandleCosimFile},
+    {"--dcachetrace", &SimulatorRPCChannel::HandleDCacheTrace},
+    {"--dsp_clock", &SimulatorRPCChannel::HandleDSPClock},
+    {"-E", &SimulatorRPCChannel::HandleSimErr},
+    {"--etm_base", &SimulatorRPCChannel::HandleETMCFGBase},
+    {"--etmcfg_base", &SimulatorRPCChannel::HandleETMCFGBase},
+    {"--gdbserv", &SimulatorRPCChannel::HandleGDBServ},
+    {"-G", &SimulatorRPCChannel::HandleGDBServ},
+    {"--hvx_length", &SimulatorRPCChannel::HandleHVXLength},
+    {"--icachetrace", &SimulatorRPCChannel::HandleICacheTrace},
+    {"-I", &SimulatorRPCChannel::HandleSimIn},
+    {"--l2cachetrace", &SimulatorRPCChannel::HandleL2CacheTrace},
+    {"--l2cfg_base", &SimulatorRPCChannel::HandleL2CFGBase},
+    {"--l2tcm_base", &SimulatorRPCChannel::HandleL2TCMBase},
+    {"--memfill", &SimulatorRPCChannel::HandleMemFill},
+    {"--memfill_rand", &SimulatorRPCChannel::HandleMemFillRand},
+    {"--memtrace", &SimulatorRPCChannel::HandleMemTrace},
+    {"-m", &SimulatorRPCChannel::HandleMemTrace},
+    {"--nullptr", &SimulatorRPCChannel::HandleNullPtr},
+    {"-O", &SimulatorRPCChannel::HandleSimOut},
+    {"--packet_analyze", &SimulatorRPCChannel::HandlePacketAnalyze},
+    {"--pcfilter", &SimulatorRPCChannel::HandlePCFilter},
+    {"--pctrace", &SimulatorRPCChannel::HandlePCTrace},
+    {"--pctrace_min", &SimulatorRPCChannel::HandlePCTraceMin},
+    {"--pctrace_nano", &SimulatorRPCChannel::HandlePCTraceNano},
+    {"-p", &SimulatorRPCChannel::HandleProfile},
+    {"--pmu_statsfile", &SimulatorRPCChannel::HandlePMUStatsFile},
+    {"--profile", &SimulatorRPCChannel::HandleProfile},
+    {"--profile_timezero", &SimulatorRPCChannel::HandleProfileTimeZero},
+    {"-q", &SimulatorRPCChannel::HandleQuiet},
+    {"--quiet", &SimulatorRPCChannel::HandleQuiet},
+    {"--reconnect", &SimulatorRPCChannel::HandleReconnect},
+    {"--rtos", &SimulatorRPCChannel::HandleRTOS},
+    {"-S", &SimulatorRPCChannel::HandleStatsFile},
+    {"--sim_err", &SimulatorRPCChannel::HandleSimErr},
+    {"--sim_in", &SimulatorRPCChannel::HandleSimIn},
+    {"--sim_out", &SimulatorRPCChannel::HandleSimOut},
+    {"--stackstart", &SimulatorRPCChannel::HandleStackStart},
+    {"--stalltrace", &SimulatorRPCChannel::HandleStallTrace},
+    {"--statsfile", &SimulatorRPCChannel::HandleStatsFile},
+    {"--subsystem_base", &SimulatorRPCChannel::HandleSubsystemBase},
+    {"--symfile", &SimulatorRPCChannel::HandleSymFile},
+    {"--tcm", &SimulatorRPCChannel::HandleTCM},
+    {"--tcm:highaddr", &SimulatorRPCChannel::HandleTCMHighAddr},
+    {"--tcm:lowaddr", &SimulatorRPCChannel::HandleTCMLowAddr},
+    {"-t", &SimulatorRPCChannel::HandlePCTrace},
+    {"--timefilter_ns", &SimulatorRPCChannel::HandleTimeFilterNS},
+    {"--timing", &SimulatorRPCChannel::HandleTiming},
+    {"--uarchtrace", &SimulatorRPCChannel::HandleUArchTrace},
+    {"-u", &SimulatorRPCChannel::HandlePCTraceMin},
+    {"--usefs", &SimulatorRPCChannel::HandleUseFS},
+    {"--v2p_translation", &SimulatorRPCChannel::HandleV2PTranslation},
+    {"--verbose", &SimulatorRPCChannel::HandleVerbose},
+};
+
+std::string SimulatorRPCChannel::Status_::str() const {
+  switch (s) {
+    case HEX_STAT_ERROR:
+      return "HEX_STAT_ERROR";
+    case HEX_STAT_SUCCESS:
+      return "HEX_STAT_SUCCESS";
+    case HEX_STAT_CANNOT_CONFIG:
+      return "HEX_STAT_CANNOT_CONFIG";
+    case HEX_STAT_INVALID_ARGS:
+      return "HEX_STAT_INVALID_ARGS";
+    case HEX_STAT_RANGE_ERROR:
+      return "HEX_STAT_RANGE_ERROR";
+    case HEX_STAT_FILE_ACCESS_ERROR:
+      return "HEX_STAT_FILE_ACCESS_ERROR";
+    case HEX_STAT_DEVICE_NOT_FOUND:
+      return "HEX_STAT_DEVICE_NOT_FOUND";
+    case HEX_STAT_MEM_ACCESS_ERROR:
+      return "HEX_STAT_MEM_ACCESS_ERROR";
+    case HEX_STAT_CANNOT_TRANSLATE:
+      return "HEX_STAT_CANNOT_TRANSLATE";
+    case HEX_STAT_NO_ACTIVE_THREADS:
+      return "HEX_STAT_NO_ACTIVE_THREADS";
+    case HEX_STAT_LOAD_ELF_ERROR:
+      return "HEX_STAT_LOAD_ELF_ERROR";
+    case HEX_STAT_CORE_RESET:
+      return "HEX_STAT_CORE_RESET";
+    default:
+      break;
+  }
+  return std::to_string(static_cast<int>(s));
+}
+
+std::string SimulatorRPCChannel::Core_::str() const {
+  switch (s) {
+    case HEX_CORE_SUCCESS:
+      return "HEX_CORE_SUCCESS";
+    case HEX_CORE_FINISHED:
+      return "HEX_CORE_FINISHED";
+    case HEX_CORE_RESET:
+      return "HEX_CORE_RESET";
+    case HEX_CORE_BREAKPOINT:
+      return "HEX_CORE_BREAKPOINT";
+    case HEX_CORE_ASYNCHRONOUS_BREAK:
+      return "HEX_CORE_ASYNCHRONOUS_BREAK";
+    case HEX_CORE_ERROR:
+      return "HEX_CORE_ERROR";
+    default:
+      break;
+  }
+  return std::to_string(static_cast<int>(s));
+}
+
+std::string SimulatorRPCChannel::Cpu_::str() const {
+  switch (c) {
+    case HEX_CPU_V65:
+      return "v65";
+    case HEX_CPU_V66:
+      return "v66";
+    case HEX_CPU_V68:
+      return "v68";
+    case HEX_CPU_V69:
+      return "v69";
+    default:
+      break;
+  }
+  return default_cpu_;
+}
+
+SimulatorRPCChannel::SDKInfo_::SDKInfo_(const std::string& sdk_root, const std::string& cpu)
+    : root(sdk_root) {
+  qurt_root = root + "/rtos/qurt/compute" + cpu;
+  runelf = qurt_root + "/sdksim_bin/runelf.pbn";
+
+  // The "run_main_on_hexagon_sim" binary lives in a subdirectory that looks
+  // like "[...]on_hexagon/ship/hexagon_toolv84_v68/run_main_on_hexagon_sim".
+  // We need to get the right "hexagon_toolv..." component, based mostly on
+  // the cpu version.
+  std::vector<std::string> dir_names;
+
+  DIR* dir = opendir((root + "/libs/run_main_on_hexagon/ship").c_str());
+  ICHECK(dir != nullptr) << "Cannot read directory " << root + "/libs/run_main_on_hexagon/ship";
+  while (dirent* d = readdir(dir)) {
+    if (d->d_type != DT_DIR) continue;
+
+    std::string name = d->d_name;
+    // Note: The first substr is always safe, and the second only executes
+    // when "name" is at least 13 characters long.
+    if (name.substr(0, 13) == "hexagon_toolv" && name.substr(name.size() - 3, 3) == cpu) {
+      dir_names.push_back(name);
+    }
+  }
+  closedir(dir);
+  ICHECK(!dir_names.empty());
+
+  auto max_it = std::max_element(dir_names.begin(), dir_names.end());
+  runmain = root + "/libs/run_main_on_hexagon/ship/" + *max_it + "/run_main_on_hexagon_sim";
+}
+
+HEX_8u_t SimulatorRPCChannel::PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV,
+                                                   HEX_8u_t RttV, HEX_8u_t RxxV, HEX_1u_t imm) {
+  // Rssv = combine(&message_buffer, &dispatch)
+  auto* rpc = reinterpret_cast<SimulatorRPCChannel*>(handle);
+  rpc->dispatch_v_ = RssV & ~0u;  // ~0u is uint32_t
+  rpc->message_buffer_v_ = RssV >> 32;
+
+  LOG(INFO) << "dispatch:" << reinterpret_cast<void*>(rpc->dispatch_v_)

Review comment:
       `DLOG` or `VLOG`




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

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

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



[GitHub] [tvm] masahi commented on a change in pull request #10361: [Hexagon] RPC server/client for simulator

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



##########
File path: src/runtime/hexagon/rpc/simulator/session.cc
##########
@@ -0,0 +1,1322 @@
+/*
+ * 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.
+ */
+
+#include <HexagonWrapper.h>
+#include <dmlc/optional.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+// POSIX includes
+#include <dirent.h>
+#include <unistd.h>
+
+#include <algorithm>
+#include <cstdlib>
+#include <deque>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <utility>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "hexagon_sim_proto.h"
+
+#define CHECKED_CALL(func, ...)                                               \
+  do {                                                                        \
+    HEXAPI_Status s = sim_->func(__VA_ARGS__);                                \
+    ICHECK_EQ(s, HEX_STAT_SUCCESS)                                            \
+        << self_name_ << ": " #func " failed with code " << Status_{s}.str(); \
+  } while (false)
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+using string_list = std::deque<std::string>;
+
+namespace detail {
+
+// An "Optional" class, originally a replacement for llvm::Optional, then an
+// extension of dmlc::optional to make it compatible with C++17's std::optional.
+template <typename T>
+struct Optional : public dmlc::optional<T> {
+  using dmlc::optional<T>::optional;
+  using dmlc::optional<T>::operator=;
+  Optional(const T& val) : dmlc::optional<T>(val) {}  // NOLINT(*)
+
+  T* operator->() { return &this->operator*(); }
+  const T* operator->() const { return &this->operator*(); }
+};
+
+// Replacement for llvm::StringSwitch.
+template <typename T>
+class StringSwitch {
+ public:
+  explicit StringSwitch(const std::string& key) : key(key) {}
+  operator T() const {
+    auto f = map.find(key);
+    if (f != map.end()) {
+      return f->second;
+    }
+    ICHECK(static_cast<bool>(def_val)) << "default value not set";
+    return *def_val;
+  }
+  StringSwitch& Case(const std::string& key, T val) {
+    map.insert(std::make_pair(key, val));
+    return *this;
+  }
+  StringSwitch& Default(T val) {
+    ICHECK(!static_cast<bool>(def_val)) << "default value already set";
+    def_val = val;
+    return *this;
+  }
+
+ private:
+  const std::string key;
+  std::map<std::string, T> map;
+  Optional<T> def_val;
+};
+
+using MaybeString = Optional<std::string>;
+
+MaybeString front(const string_list& deq) {
+  return !deq.empty() ? MaybeString(deq.front()) : MaybeString();
+}
+
+MaybeString pop_front(string_list& deq) {  // NOLINT(*)
+  if (deq.empty()) return MaybeString();
+  std::string f = deq.front();
+  deq.pop_front();
+  return MaybeString(f);
+}
+
+// Functions used when parsing the argument string.
+
+Optional<int64_t> to_int(const MaybeString& str) {
+  auto none = Optional<int64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      int64_t val = std::stoll(*str, &pos, 0);
+      return pos == str->size() ? Optional<int64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<uint64_t> to_uint(const MaybeString& str) {
+  auto none = Optional<uint64_t>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      uint64_t val = std::stoull(*str, &pos, 0);
+      return pos == str->size() ? Optional<uint64_t>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<float> to_float(const MaybeString& str) {
+  auto none = Optional<float>();
+  if (str.has_value()) {
+    try {
+      size_t pos;
+      float val = std::stof(*str, &pos);
+      return pos == str->size() ? Optional<float>(val) : none;
+    } catch (std::invalid_argument&) {
+    }
+  }
+  return none;
+}
+
+Optional<bool> to_bool(const MaybeString& str) {
+  auto none = Optional<bool>();
+  if (auto num = to_int(str)) {
+    if (*num == 0) return false;
+    if (*num == 1) return true;
+    return none;
+  }
+  if (str) {
+    if (*str == "true" || *str == "TRUE") return true;
+    if (*str == "false" || *str == "FALSE") return false;
+  }
+  return none;
+}
+
+template <typename T>
+using MaybeRange = Optional<std::pair<T, T>>;
+
+template <typename T, Optional<T> Parse(const MaybeString&)>
+MaybeRange<T> to_range(const MaybeString& str) {
+  auto none = MaybeRange<T>();
+  if (str && !str->empty()) {
+    auto n = str->find('-', 1);
+    if (n != std::string::npos) {
+      auto begin = Parse(str->substr(0, n));
+      auto end = Parse(str->substr(n + 1, str->size() - n - 1));
+      if (begin && end) {
+        return std::make_pair(*begin, *end);
+      }
+    }
+  }
+  return none;
+}
+
+}  // namespace detail
+
+class SimulatorRPCChannel final : public RPCChannel {
+ public:
+  SimulatorRPCChannel(std::string args);
+  ~SimulatorRPCChannel() final;
+  size_t Send(const void* data, size_t size) final;
+  size_t Recv(void* data, size_t size) final;
+
+ private:
+  struct Status_ {
+    HEXAPI_Status s;
+    std::string str() const;
+  };
+  struct Core_ {
+    HEXAPI_CoreState s;
+    std::string str() const;
+  };
+  struct Cpu_ {
+    HEXAPI_Cpu c;
+    std::string str() const;
+  };
+  struct SDKInfo_ {
+    SDKInfo_(const std::string& sdk_root, const std::string& cpu);
+    std::string root;
+    std::string qurt_root;  // sdk_root/rtos/qurt/computevNN.
+    std::string runelf;     // Path to runelf.pbn.
+    std::string runmain;    // Path to run_main_on_hexagon.
+  };
+
+  Message SendMsg(Message msg);
+  Message SendMsg(uint32_t code, uint32_t len, uint32_t va);
+  void ReadFromProcess(void* host_dst, HEX_VA_t src, size_t len);
+  void WriteToProcess(HEX_VA_t dst, const void* host_src, size_t len);
+
+  static HEX_8u_t PassVirtAddrCallback(void* handle, int threadno, HEX_8u_t RssV, HEX_8u_t RttV,
+                                       HEX_8u_t RxxV, HEX_1u_t imm);
+
+  detail::Optional<HEXAPI_Cpu> GetCPU(const detail::MaybeString& cpu_str);
+
+  // File name templates for mkstemps.
+#define SUFFIX ".cfg"
+  static constexpr int template_length_ = strlen("temp-xxxx-XXXXXX" SUFFIX) + 1;
+  char osam_file_[template_length_] = "temp-osam-XXXXXX" SUFFIX;
+  char cosim_file_[template_length_] = "temp-q6ss-XXXXXX" SUFFIX;
+  const int suffix_len_ = strlen(SUFFIX);
+#undef SUFFIX
+
+  static const constexpr char* self_name_ = "SimulatorRPCChannel";
+  static const constexpr char* default_cpu_ = "v68";
+  std::string cpu_;
+
+  HEX_VA_t dispatch_v_, message_buffer_v_;
+  std::unique_ptr<HexagonWrapper> sim_;
+
+  // Sim configuration routines.

Review comment:
       @mehrdadh Since the original code will be removed, I think it's fine to merge this as is and let @kparzysz-quic continue completing his work.




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

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

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



[GitHub] [tvm] kparzysz-quic commented on pull request #10361: [Hexagon] RPC server/client for simulator

Posted by GitBox <gi...@apache.org>.
kparzysz-quic commented on pull request #10361:
URL: https://github.com/apache/tvm/pull/10361#issuecomment-1048900971


   cc: @mehrdadh 


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

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

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



[GitHub] [tvm] kparzysz-quic merged pull request #10361: [Hexagon] RPC server/client for simulator

Posted by GitBox <gi...@apache.org>.
kparzysz-quic merged pull request #10361:
URL: https://github.com/apache/tvm/pull/10361


   


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

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

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