You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/12/01 19:46:52 UTC

[GitHub] [tvm] mehrdadh opened a new pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

mehrdadh opened a new pull request #9631:
URL: https://github.com/apache/tvm/pull/9631


   This PR:
   - Reuses minRPC implementation for hexagon to create RPC connection from host to hexagon device which is passed through android RPC server.
   - Adds HexagonLauncher which manages file upload to Android, initiate RPC server on Android and manage RPC session to hexagon.
   
   To use this feature you need to build TVM using these configs:
   
   ```
   cmake -DUSE_HEXAGON_RPC=ON \
          -DUSE_ANDROID_TOOLCHAIN=/path/to/android-ndk/build/cmake/android.toolchain.cmake \
          -DANDROID_PLATFORM=android-28 \
          -DANDROID_ABI=arm64-v8a \
          -DUSE_HEXAGON_ARCH=v65|v66|v68 \
          -DUSE_HEXAGON_SDK=/path/to/Hexagon/SDK \
          -DUSE_HEXAGON_TOOLCHAIN=/path/to/Hexagon/toolchain/ \
          -DUSE_LLVM=/path/to/llvm/bin/llvm-config \
          -DUSE_CPP_RPC=ON \
          -DCMAKE_CXX_COMPILER=/path/to/clang++ \    
          -DCMAKE_CXX_FLAGS='-stdlib=libc++' ..
   ```
   
   This will build:
   - `libtvm` and `libtvm_runtime` using hexagon LLVM and`tvm_rcp` for **host**.
   - `libtvm_runtime_android.so` and `tvm_rpc_android` for **Android** under `build/hexagon_rpc`
   - `libhexagon_rpc_skel.so` library which include RPC server for **Hexagon** under `build/hexagon_rpc`
   
   cc @areusch @csullivan @adstraw


-- 
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] csullivan commented on a change in pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024

Review comment:
       TODO to make this configurable.




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)
+      : read_buffer_{read_buffer},
+        read_buffer_size_bytes_{read_buffer_size_bytes},
+        read_buffer_index_{0} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d",
+                  write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      HEXAGON_PRINT(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+                  read_buffer_index_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_index_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_index_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_index_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      HEXAGON_PRINT(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read,
+                    read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  /*!
+   * \brief Set read buffer in IOHandler to data pointer.
+   * \param data The data pointer.
+   * \param data_size_bytes The size of data in bytes.
+   *
+   * \return The status
+   */
+  AEEResult SetReadBuffer(const uint8_t* data, size_t data_size_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_index_: ",
+                  data_size_bytes, read_buffer_index_);
+    if (data_size_bytes > read_buffer_size_bytes_) {
+      return AEE_EFAILED;
+    }
+    read_buffer_ = data;
+    read_buffer_index_ = data_size_bytes;
+    return AEE_SUCCESS;
+  }
+
+  /*!
+   * \brief Get pointer to the buffer that a packet has been written to.

Review comment:
       This function doesn't "get any pointers", it copies (part of) the write buffer to the specified address.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)
+      : read_buffer_{read_buffer},
+        read_buffer_size_bytes_{read_buffer_size_bytes},
+        read_buffer_index_{0} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d",
+                  write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      HEXAGON_PRINT(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+                  read_buffer_index_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_index_ - read_len_bytes) < 0) {

Review comment:
       This shouldn't be subtracting unsigned numbers, just comparing them.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)
+      : read_buffer_{read_buffer},
+        read_buffer_size_bytes_{read_buffer_size_bytes},
+        read_buffer_index_{0} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d",
+                  write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      HEXAGON_PRINT(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+                  read_buffer_index_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_index_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_index_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_index_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      HEXAGON_PRINT(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read,
+                    read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  /*!
+   * \brief Set read buffer in IOHandler to data pointer.
+   * \param data The data pointer.
+   * \param data_size_bytes The size of data in bytes.
+   *
+   * \return The status
+   */
+  AEEResult SetReadBuffer(const uint8_t* data, size_t data_size_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_index_: ",
+                  data_size_bytes, read_buffer_index_);
+    if (data_size_bytes > read_buffer_size_bytes_) {
+      return AEE_EFAILED;
+    }
+    read_buffer_ = data;
+    read_buffer_index_ = data_size_bytes;
+    return AEE_SUCCESS;
+  }
+
+  /*!
+   * \brief Get pointer to the buffer that a packet has been written to.
+   * \param buf The data pointer.
+   * \param read_size_bytes The size of read in bytes.
+   *
+   * \return The size of data that is read in bytes.
+   */
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_size_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d",
+                  read_size_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_size_bytes);
+  }
+
+  void Close() { HEXAGON_PRINT(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_index_;
+  size_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer, size_t receive_buffer_size_bytes)
+      : io_{receive_buffer, receive_buffer_size_bytes}, rpc_server_{&io_} {};
+
+  /*!
+   * \brief Wrtie to IOHandler.
+   * \param data The data pointer
+   * \param data_size_bytes The data size in bytes.
+   *
+   * \return The size of data written to IOHandler.
+   */
+  int64_t Write(const uint8_t* data, size_t data_size_bytes) {
+    if (io_.SetReadBuffer(data, data_size_bytes) != AEE_SUCCESS) {
+      return -1;
+    }
+    rpc_server_.ProcessOnePacket();
+    return (int64_t)data_size_bytes;
+  }
+
+  /*!
+   * \brief Read from IOHandler.
+   * \param buf The buffer pointer
+   * \param read_size_bytes Read request size in bytes.
+   *
+   * \return The size of data that is read in bytes.
+   */
+  int64_t Read(uint8_t* buf, size_t read_size_bytes) {
+    return io_.GetWriteBuffer(buf, read_size_bytes);
+  }
+
+ private:
+  HexagonIOHandler io_;
+  MinRPCServer<HexagonIOHandler> rpc_server_;
+};
+
+}  // namespace hexagon
+}  // namespace runtime
+}  // namespace tvm
+
+namespace {
+tvm::runtime::hexagon::HexagonRPCServer* get_hexagon_rpc_server() {
+  static tvm::runtime::hexagon::HexagonRPCServer g_hexagon_rpc_server(
+      new uint8_t[TVM_HEXAGON_RPC_BUFF_SIZE_BYTES], TVM_HEXAGON_RPC_BUFF_SIZE_BYTES);

Review comment:
       This allocation is lost the first time the server tries to read something (see `SetReadBuffer`).

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)
+      : read_buffer_{read_buffer},
+        read_buffer_size_bytes_{read_buffer_size_bytes},
+        read_buffer_index_{0} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d",
+                  write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      HEXAGON_PRINT(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+                  read_buffer_index_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_index_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_index_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_index_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      HEXAGON_PRINT(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read,
+                    read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  /*!
+   * \brief Set read buffer in IOHandler to data pointer.
+   * \param data The data pointer.
+   * \param data_size_bytes The size of data in bytes.
+   *
+   * \return The status
+   */
+  AEEResult SetReadBuffer(const uint8_t* data, size_t data_size_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_index_: ",
+                  data_size_bytes, read_buffer_index_);
+    if (data_size_bytes > read_buffer_size_bytes_) {
+      return AEE_EFAILED;
+    }
+    read_buffer_ = data;

Review comment:
       This loses the initial read buffer, which is never deallocated.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)
+      : read_buffer_{read_buffer},
+        read_buffer_size_bytes_{read_buffer_size_bytes},
+        read_buffer_index_{0} {}

Review comment:
       The order here should follow the order of declarations below.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)
+      : read_buffer_{read_buffer},
+        read_buffer_size_bytes_{read_buffer_size_bytes},
+        read_buffer_index_{0} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d",
+                  write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      HEXAGON_PRINT(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    HEXAGON_PRINT(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+                  read_buffer_index_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_index_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_index_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_index_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      HEXAGON_PRINT(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read,
+                    read_len_bytes);

Review comment:
       If it's an error to read less than `read_len_bytes`, then why is the code above trying to handle this?

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+// TODO(mehrdadh): make this configurable.
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 2 * 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+/*!
+ * \brief Hexagon IO Handler used in HexagonRPCServer(MinRPCServer).
+ *
+ * \param read_buffer The pointer to read buffer.
+ * \param read_buffer_size_bytes The read buffer size in bytes.
+ */
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer, size_t read_buffer_size_bytes)

Review comment:
       The `read_buffer` is not used for anything.




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);
+    rpc_server_.ProcessOnePacket();
+    return (int64_t)data_len_bytes;
+  }
+
+  int64_t Read(uint8_t* buf, size_t read_len_bytes) {
+    return io_.GetWriteBuffer(buf, read_len_bytes);
+  }
+
+ private:
+  HexagonIOHandler io_;
+  MinRPCServer<HexagonIOHandler> rpc_server_;
+};
+
+}  // namespace hexagon
+}  // namespace runtime
+}  // namespace tvm
+
+static tvm::runtime::hexagon::HexagonRPCServer* g_hexagon_rpc_server = nullptr;

Review comment:
       can you clarify this?




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -241,10 +340,39 @@ if (USE_HEXAGON_DEVICE STREQUAL "${PICK_NONE}")
   elseif(BUILD_FOR_ANDROID AND HEXAGON_SDK_PATH_DEFINED)
     list(APPEND RUNTIME_HEXAGON_SRCS src/runtime/hexagon/proxy_rpc/device_api.cc)
   else()
-    file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
+  file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
   endif()
 else()
   file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/android/*.cc)
 endif()
+
+if(USE_HEXAGON_RPC)
+  file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
+endif()
+
+if(USE_HEXAGON_SDK AND BUILD_FOR_ANDROID)
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+  include_directories(SYSTEM ${HEXAGON_SDK_INCLUDES} ${HEXAGON_REMOTE_ROOT})
+
+  set(QAIC_EXE "${HEXAGON_QAIC_EXE}")
+  foreach(INCDIR IN LISTS HEXAGON_SDK_INCLUDES HEXAGON_REMOTE_ROOT)
+    list(APPEND QAIC_FLAGS "-I${INCDIR}")
+  endforeach()
+
+  set(HEXAGON_RPC_DIR "${CMAKE_SOURCE_DIR}/src/runtime/hexagon/rpc")
+  set(RPC_IDL "hexagon_rpc.idl")
+  set(RPC_H "hexagon_rpc.h")
+  set(RPC_STUB_C "hexagon_rpc_stub.c")
+  
+  add_custom_command(
+    OUTPUT "${HEXAGON_RPC_DIR}/${RPC_STUB_C}" "${HEXAGON_RPC_DIR}/${RPC_H}"
+    COMMAND ${QAIC_EXE} ${QAIC_FLAGS} "${HEXAGON_RPC_DIR}/${RPC_IDL}" -o ${HEXAGON_RPC_DIR}
+    MAIN_DEPENDENCY "${HEXAGON_RPC_DIR}/${RPC_IDL}"
+  )
+  file(GLOB HEXAGON_RPC_CPP "${HEXAGON_RPC_DIR}/android/*.cc")

Review comment:
       This will unconditionally add all files from that directory to the runtime target.  If `USE_HEXAGON_RPC` is not set, it will cause link errors.




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);
+    rpc_server_.ProcessOnePacket();
+    return (int64_t)data_len_bytes;
+  }
+
+  int64_t Read(uint8_t* buf, size_t read_len_bytes) {
+    return io_.GetWriteBuffer(buf, read_len_bytes);
+  }
+
+ private:
+  HexagonIOHandler io_;
+  MinRPCServer<HexagonIOHandler> rpc_server_;
+};
+
+}  // namespace hexagon
+}  // namespace runtime
+}  // namespace tvm
+
+static tvm::runtime::hexagon::HexagonRPCServer* g_hexagon_rpc_server = nullptr;

Review comment:
       addressed here: https://github.com/apache/tvm/pull/9631/commits/31fda2bd91f0f73b99bde0b85f32180b9404c58a

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);
+    rpc_server_.ProcessOnePacket();
+    return (int64_t)data_len_bytes;
+  }
+
+  int64_t Read(uint8_t* buf, size_t read_len_bytes) {
+    return io_.GetWriteBuffer(buf, read_len_bytes);
+  }
+
+ private:
+  HexagonIOHandler io_;
+  MinRPCServer<HexagonIOHandler> rpc_server_;
+};
+
+}  // namespace hexagon
+}  // namespace runtime
+}  // namespace tvm
+
+static tvm::runtime::hexagon::HexagonRPCServer* g_hexagon_rpc_server = nullptr;
+
+static AEEResult hexagon_rpc_server_init() {
+  uint8_t* receive_buffer = new uint8_t[TVM_HEXAGON_RPC_BUFF_SIZE_BYTES];

Review comment:
       I added check for buffer size and passed the size to IOHandler:
   https://github.com/apache/tvm/pull/9631/commits/31fda2bd91f0f73b99bde0b85f32180b9404c58a




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -241,10 +340,39 @@ if (USE_HEXAGON_DEVICE STREQUAL "${PICK_NONE}")
   elseif(BUILD_FOR_ANDROID AND HEXAGON_SDK_PATH_DEFINED)
     list(APPEND RUNTIME_HEXAGON_SRCS src/runtime/hexagon/proxy_rpc/device_api.cc)
   else()
-    file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
+  file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
   endif()
 else()
   file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/android/*.cc)
 endif()
+
+if(USE_HEXAGON_RPC)
+  file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
+endif()
+
+if(USE_HEXAGON_SDK AND BUILD_FOR_ANDROID)
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+  include_directories(SYSTEM ${HEXAGON_SDK_INCLUDES} ${HEXAGON_REMOTE_ROOT})
+
+  set(QAIC_EXE "${HEXAGON_QAIC_EXE}")
+  foreach(INCDIR IN LISTS HEXAGON_SDK_INCLUDES HEXAGON_REMOTE_ROOT)
+    list(APPEND QAIC_FLAGS "-I${INCDIR}")
+  endforeach()
+
+  set(HEXAGON_RPC_DIR "${CMAKE_SOURCE_DIR}/src/runtime/hexagon/rpc")
+  set(RPC_IDL "hexagon_rpc.idl")
+  set(RPC_H "hexagon_rpc.h")
+  set(RPC_STUB_C "hexagon_rpc_stub.c")
+  
+  add_custom_command(
+    OUTPUT "${HEXAGON_RPC_DIR}/${RPC_STUB_C}" "${HEXAGON_RPC_DIR}/${RPC_H}"
+    COMMAND ${QAIC_EXE} ${QAIC_FLAGS} "${HEXAGON_RPC_DIR}/${RPC_IDL}" -o ${HEXAGON_RPC_DIR}
+    MAIN_DEPENDENCY "${HEXAGON_RPC_DIR}/${RPC_IDL}"
+  )
+  file(GLOB HEXAGON_RPC_CPP "${HEXAGON_RPC_DIR}/android/*.cc")

Review comment:
       @kparzysz-quic happy to hear your suggestion on this. thanks!




-- 
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] csullivan commented on a change in pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]

Review comment:
       In my experiments I've not needed to setup up port forwarding and reversing for the RPC server to connect to the tracker. I was able to remove these in your code and run the tests okay. That may be because the adb session already had these forwarded and reversed. That's perhaps another reason why I don't like making port forward/reversing changes as they change the state of the adb session even after the tests complete. 




-- 
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] jroesch merged pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

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


   


-- 
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] csullivan commented on a change in pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: apps/cpp_rpc/CMakeLists.txt
##########
@@ -43,4 +45,12 @@ target_include_directories(
   PUBLIC DMLC_PATH
 )
 
-target_link_libraries(tvm_rpc tvm_runtime)
+if (BUILD_FOR_ANDROID AND USE_HEXAGON_SDK)
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+  link_directories(${HEXAGON_REMOTE_ROOT})
+  list(APPEND TVM_RPC_LINKER_LIBS cdsprpc log)
+endif()

Review comment:
       I'm okay with conditionally linking against the hexagon sdk if we can't get around this for now. There exists precedent on custom build behavior for other sdks (opencl). Would be great to loop back and remove this in the future if possible.




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

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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -202,6 +200,108 @@ if(USE_HEXAGON_PROXY_RPC STREQUAL "ON")
   set_directory_properties(PROPERTIES ADDITIONAL_MAKE_CLEAN_FILES "${RPC_BINARY_DIR}")
 endif()
 
+if(USE_HEXAGON_RPC)
+  if(DEFINED USE_ANDROID_TOOLCHAIN)
+    if(NOT DEFINED ANDROID_PLATFORM)
+      message(SEND_ERROR "Please set ANDROID_PLATFORM "
+        "when providing an Android cmake toolchain.")
+    endif()
+    if(NOT DEFINED ANDROID_ABI)
+      message(SEND_ERROR "Please set ANDROID_ABI "
+        "when providing an Android cmake toolchain.")
+    endif()
+  else()
+    message(SEND_ERROR "Please set USE_ANDROID_TOOLCHAIN to build the android "
+      "RPC server for Hexagon.")
+  endif()
+
+  if(NOT DEFINED USE_HEXAGON_SDK)
+    message(SEND_ERROR "Please set USE_HEXAGON_SDK to build the android "
+      "RPC server for Hexagon RPC.")
+  endif()
+  if(NOT DEFINED USE_HEXAGON_ARCH)
+    message(SEND_ERROR "Please set USE_HEXAGON_ARCH to build the android "
+      "RPC server for Hexagon RPC.")
+  endif()
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+
+  set(HEXAGON_RPC_OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/hexagon_rpc")
+  file(MAKE_DIRECTORY ${HEXAGON_RPC_OUTPUT})
+
+  # Android Part
+  ExternalProject_Add(android_runtime_rpc
+    SOURCE_DIR "${CMAKE_SOURCE_DIR}"
+    BUILD_COMMAND $(MAKE) runtime tvm_rpc
+    CMAKE_ARGS
+    "-DCMAKE_TOOLCHAIN_FILE=${USE_ANDROID_TOOLCHAIN}"
+    "-DUSE_ANDROID_TOOLCHAIN=${USE_ANDROID_TOOLCHAIN}"
+    "-DANDROID_PLATFORM=${ANDROID_PLATFORM}"
+    "-DANDROID_ABI=${ANDROID_ABI}"
+    "-DCMAKE_CXX_STANDARD=14"
+    "-DUSE_LIBBACKTRACE=OFF"
+    "-DUSE_LLVM=OFF"
+    "-DUSE_RPC=ON"
+    "-DUSE_CPP_RPC=ON"
+    "-DUSE_HEXAGON_SDK=${USE_HEXAGON_SDK}"
+    "-DUSE_HEXAGON_ARCH=${USE_HEXAGON_ARCH}"
+    "-DCMAKE_VERBOSE_MAKEFILE=ON"
+    INSTALL_COMMAND ""
+    BUILD_ALWAYS ON
+  )
+  ExternalProject_Get_Property(android_runtime_rpc BINARY_DIR)
+  ExternalProject_Add_Step(android_runtime_rpc copy_binary_runtime
+    COMMAND ${CMAKE_COMMAND} -E copy_if_different
+      ${BINARY_DIR}/libtvm_runtime.so
+      ${HEXAGON_RPC_OUTPUT}/libtvm_runtime_android.so
+    DEPENDEES install
+  )
+  ExternalProject_Add_Step(android_runtime_rpc copy_binary_rpc
+    COMMAND ${CMAKE_COMMAND} -E copy_if_different
+      ${BINARY_DIR}/tvm_rpc
+      ${HEXAGON_RPC_OUTPUT}/tvm_rpc_android
+    DEPENDEES install
+  )
+
+  if("${USE_HEXAGON_TOOLCHAIN}" STREQUAL "")
+    message(SEND_ERROR "Please set USE_HEXAGON_TOOLCHAIN to build the hexagon "
+      "RPC SKEL.")
+  endif()
+  find_hexagon_toolchain()
+  message(STATUS "HEXAGON_TOOLCHAIN: ${HEXAGON_TOOLCHAIN}")
+
+  # Hexagon Part
+  ExternalProject_Add(hexagon_rpc_skel
+    SOURCE_DIR "${CMAKE_SOURCE_DIR}/cmake/libs/hexagon_rpc_skel"
+    INSTALL_DIR "${LAUNCHER_BINARY_DIR}"
+    CMAKE_ARGS
+    "-DCMAKE_C_COMPILER=${HEXAGON_TOOLCHAIN}/bin/hexagon-clang"
+    "-DCMAKE_CXX_COMPILER=${HEXAGON_TOOLCHAIN}/bin/hexagon-clang++"
+    "-DFASTRPC_LIBS=SKEL"
+    "-DUSE_HEXAGON_ARCH=${USE_HEXAGON_ARCH}"
+    "-DUSE_HEXAGON_SDK=${USE_HEXAGON_SDK}"
+    INSTALL_COMMAND ""
+    BUILD_ALWAYS ON
+  )
+  ExternalProject_Get_Property(hexagon_rpc_skel BINARY_DIR)
+  ExternalProject_Add_Step(hexagon_rpc_skel copy_hexagon_skel
+    COMMAND ${CMAKE_COMMAND} -E copy_if_different
+      ${BINARY_DIR}/libhexagon_rpc_skel.so
+      ${HEXAGON_RPC_OUTPUT}/libhexagon_rpc_skel.so
+    DEPENDEES install
+  )
+
+  # copy android_bash template file
+  configure_file("${CMAKE_SOURCE_DIR}/src/runtime/hexagon/rpc/android_bash.sh.template" 
+    ${HEXAGON_RPC_OUTPUT} COPYONLY)
+endif()

Review comment:
       done here: https://github.com/apache/tvm/pull/9631/commits/28db608e66b16ffb8159113d2edd4408a90caa89

##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -217,16 +317,20 @@ if(USE_HEXAGON_DEVICE STREQUAL "${PICK_SIM}")
       "-DHEXAGON_ARCH=${USE_HEXAGON_ARCH}"
     INSTALL_COMMAND "true"
   )
-elseif(USE_HEXAGON_DEVICE STREQUAL "${PICK_HW}")
+elseif((USE_HEXAGON_DEVICE STREQUAL "${PICK_HW}") OR (USE_HEXAGON_RPC AND BUILD_FOR_ANDROID))

Review comment:
       done: https://github.com/apache/tvm/pull/9631/commits/28db608e66b16ffb8159113d2edd4408a90caa89

##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -240,11 +344,36 @@ if (USE_HEXAGON_DEVICE STREQUAL "${PICK_NONE}")
     file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/hexagon/*.cc)
   elseif(BUILD_FOR_ANDROID AND HEXAGON_SDK_PATH_DEFINED)
     list(APPEND RUNTIME_HEXAGON_SRCS src/runtime/hexagon/proxy_rpc/device_api.cc)
-  else()
+  elseif(USE_HEXAGON_RPC)

Review comment:
       done: https://github.com/apache/tvm/pull/9631/commits/28db608e66b16ffb8159113d2edd4408a90caa89




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]

Review comment:
       we do need to do that. Otherwise, the RPC Server on android cannot connect to RPC tracker on host




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -241,10 +340,39 @@ if (USE_HEXAGON_DEVICE STREQUAL "${PICK_NONE}")
   elseif(BUILD_FOR_ANDROID AND HEXAGON_SDK_PATH_DEFINED)
     list(APPEND RUNTIME_HEXAGON_SRCS src/runtime/hexagon/proxy_rpc/device_api.cc)
   else()
-    file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
+  file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
   endif()
 else()
   file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/android/*.cc)
 endif()
+
+if(USE_HEXAGON_RPC)
+  file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/host/*.cc)
+endif()
+
+if(USE_HEXAGON_SDK AND BUILD_FOR_ANDROID)
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+  include_directories(SYSTEM ${HEXAGON_SDK_INCLUDES} ${HEXAGON_REMOTE_ROOT})
+
+  set(QAIC_EXE "${HEXAGON_QAIC_EXE}")
+  foreach(INCDIR IN LISTS HEXAGON_SDK_INCLUDES HEXAGON_REMOTE_ROOT)
+    list(APPEND QAIC_FLAGS "-I${INCDIR}")
+  endforeach()
+
+  set(HEXAGON_RPC_DIR "${CMAKE_SOURCE_DIR}/src/runtime/hexagon/rpc")
+  set(RPC_IDL "hexagon_rpc.idl")
+  set(RPC_H "hexagon_rpc.h")
+  set(RPC_STUB_C "hexagon_rpc_stub.c")
+  
+  add_custom_command(
+    OUTPUT "${HEXAGON_RPC_DIR}/${RPC_STUB_C}" "${HEXAGON_RPC_DIR}/${RPC_H}"
+    COMMAND ${QAIC_EXE} ${QAIC_FLAGS} "${HEXAGON_RPC_DIR}/${RPC_IDL}" -o ${HEXAGON_RPC_DIR}
+    MAIN_DEPENDENCY "${HEXAGON_RPC_DIR}/${RPC_IDL}"
+  )
+  file(GLOB HEXAGON_RPC_CPP "${HEXAGON_RPC_DIR}/android/*.cc")

Review comment:
       @kparzysz-quic thanks for pointing out. This could break if `USE_HEXAGON_SDK` and `BUILD_FOR_ANDROID` are enabled and `USE_HEXAGON_RPC` not enabled. So we should fix this. I suggest that we fix this with a cleanup of Hexagon.cmake once we agreed on using hexagon RPC and decided to deprecate `hexagon proxy rpc` and `apps/hexagon_launcher`.
   




-- 
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] csullivan commented on a change in pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -202,6 +200,108 @@ if(USE_HEXAGON_PROXY_RPC STREQUAL "ON")
   set_directory_properties(PROPERTIES ADDITIONAL_MAKE_CLEAN_FILES "${RPC_BINARY_DIR}")
 endif()
 
+if(USE_HEXAGON_RPC)
+  if(DEFINED USE_ANDROID_TOOLCHAIN)
+    if(NOT DEFINED ANDROID_PLATFORM)
+      message(SEND_ERROR "Please set ANDROID_PLATFORM "
+        "when providing an Android cmake toolchain.")
+    endif()
+    if(NOT DEFINED ANDROID_ABI)
+      message(SEND_ERROR "Please set ANDROID_ABI "
+        "when providing an Android cmake toolchain.")
+    endif()
+  else()
+    message(SEND_ERROR "Please set USE_ANDROID_TOOLCHAIN to build the android "
+      "RPC server for Hexagon.")
+  endif()
+
+  if(NOT DEFINED USE_HEXAGON_SDK)
+    message(SEND_ERROR "Please set USE_HEXAGON_SDK to build the android "
+      "RPC server for Hexagon RPC.")
+  endif()
+  if(NOT DEFINED USE_HEXAGON_ARCH)
+    message(SEND_ERROR "Please set USE_HEXAGON_ARCH to build the android "
+      "RPC server for Hexagon RPC.")
+  endif()
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+
+  set(HEXAGON_RPC_OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/hexagon_rpc")
+  file(MAKE_DIRECTORY ${HEXAGON_RPC_OUTPUT})
+
+  # Android Part
+  ExternalProject_Add(android_runtime_rpc
+    SOURCE_DIR "${CMAKE_SOURCE_DIR}"
+    BUILD_COMMAND $(MAKE) runtime tvm_rpc
+    CMAKE_ARGS
+    "-DCMAKE_TOOLCHAIN_FILE=${USE_ANDROID_TOOLCHAIN}"
+    "-DUSE_ANDROID_TOOLCHAIN=${USE_ANDROID_TOOLCHAIN}"
+    "-DANDROID_PLATFORM=${ANDROID_PLATFORM}"
+    "-DANDROID_ABI=${ANDROID_ABI}"
+    "-DCMAKE_CXX_STANDARD=14"
+    "-DUSE_LIBBACKTRACE=OFF"
+    "-DUSE_LLVM=OFF"
+    "-DUSE_RPC=ON"
+    "-DUSE_CPP_RPC=ON"
+    "-DUSE_HEXAGON_SDK=${USE_HEXAGON_SDK}"
+    "-DUSE_HEXAGON_ARCH=${USE_HEXAGON_ARCH}"
+    "-DCMAKE_VERBOSE_MAKEFILE=ON"
+    INSTALL_COMMAND ""
+    BUILD_ALWAYS ON
+  )
+  ExternalProject_Get_Property(android_runtime_rpc BINARY_DIR)
+  ExternalProject_Add_Step(android_runtime_rpc copy_binary_runtime
+    COMMAND ${CMAKE_COMMAND} -E copy_if_different
+      ${BINARY_DIR}/libtvm_runtime.so
+      ${HEXAGON_RPC_OUTPUT}/libtvm_runtime_android.so
+    DEPENDEES install
+  )
+  ExternalProject_Add_Step(android_runtime_rpc copy_binary_rpc
+    COMMAND ${CMAKE_COMMAND} -E copy_if_different
+      ${BINARY_DIR}/tvm_rpc
+      ${HEXAGON_RPC_OUTPUT}/tvm_rpc_android
+    DEPENDEES install
+  )
+
+  if("${USE_HEXAGON_TOOLCHAIN}" STREQUAL "")
+    message(SEND_ERROR "Please set USE_HEXAGON_TOOLCHAIN to build the hexagon "
+      "RPC SKEL.")
+  endif()
+  find_hexagon_toolchain()
+  message(STATUS "HEXAGON_TOOLCHAIN: ${HEXAGON_TOOLCHAIN}")
+
+  # Hexagon Part
+  ExternalProject_Add(hexagon_rpc_skel
+    SOURCE_DIR "${CMAKE_SOURCE_DIR}/cmake/libs/hexagon_rpc_skel"
+    INSTALL_DIR "${LAUNCHER_BINARY_DIR}"
+    CMAKE_ARGS
+    "-DCMAKE_C_COMPILER=${HEXAGON_TOOLCHAIN}/bin/hexagon-clang"
+    "-DCMAKE_CXX_COMPILER=${HEXAGON_TOOLCHAIN}/bin/hexagon-clang++"
+    "-DFASTRPC_LIBS=SKEL"
+    "-DUSE_HEXAGON_ARCH=${USE_HEXAGON_ARCH}"
+    "-DUSE_HEXAGON_SDK=${USE_HEXAGON_SDK}"
+    INSTALL_COMMAND ""
+    BUILD_ALWAYS ON
+  )
+  ExternalProject_Get_Property(hexagon_rpc_skel BINARY_DIR)
+  ExternalProject_Add_Step(hexagon_rpc_skel copy_hexagon_skel
+    COMMAND ${CMAKE_COMMAND} -E copy_if_different
+      ${BINARY_DIR}/libhexagon_rpc_skel.so
+      ${HEXAGON_RPC_OUTPUT}/libhexagon_rpc_skel.so
+    DEPENDEES install
+  )
+
+  # copy android_bash template file
+  configure_file("${CMAKE_SOURCE_DIR}/src/runtime/hexagon/rpc/android_bash.sh.template" 
+    ${HEXAGON_RPC_OUTPUT} COPYONLY)
+endif()

Review comment:
       Clean up artifacts, e.g.
   ```suggestion
     set_directory_properties(PROPERTIES ADDITIONAL_MAKE_CLEAN_FILES "${HEXAGON_RPC_OUTPUT}")
   endif()
   ```

##########
File path: apps/cpp_rpc/CMakeLists.txt
##########
@@ -43,4 +45,12 @@ target_include_directories(
   PUBLIC DMLC_PATH
 )
 
-target_link_libraries(tvm_rpc tvm_runtime)
+if (BUILD_FOR_ANDROID AND USE_HEXAGON_SDK)
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+  link_directories(${HEXAGON_REMOTE_ROOT})
+  list(APPEND TVM_RPC_LINKER_LIBS cdsprpc log)
+endif()

Review comment:
       libtvm_runtime.so (android) surely needs to link against the hexagon sdk libs, but I'm not clear on why you're linking tvm_rpc against them as well. 

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {

Review comment:
       Docstring that indicates this is the IOHnadler used with the MinRPCServer

##########
File path: python/tvm/contrib/hexagon/session.py
##########
@@ -0,0 +1,71 @@
+# 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 a Session class for Hexagon devices."""
+
+from tvm import rpc as _rpc
+
+
+class Session:
+    """Hexagon Device Session
+
+    Parameters
+    ----------
+    remote_kw : dict
+        Remote configs for RPC tracker.
+
+    session_name : str
+        Hexagon RPC session name.
+    """
+
+    def __init__(
+        self,
+        remote_kw: dict,
+        session_name: str = "hexagon-rpc",
+    ):
+        self._session_name = session_name
+        self._remote_kw = remote_kw
+        self._rpc = None
+        self.device = None
+
+    def __enter__(self):
+        if self.device:
+            return self

Review comment:
       else?

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);

Review comment:
       nit: HEXAGON_PRINT until we deprecate and replace all HEXAGON_PRINT with a LOG impl. 

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);
+    rpc_server_.ProcessOnePacket();
+    return (int64_t)data_len_bytes;
+  }
+
+  int64_t Read(uint8_t* buf, size_t read_len_bytes) {
+    return io_.GetWriteBuffer(buf, read_len_bytes);
+  }
+
+ private:
+  HexagonIOHandler io_;
+  MinRPCServer<HexagonIOHandler> rpc_server_;
+};
+
+}  // namespace hexagon
+}  // namespace runtime
+}  // namespace tvm
+
+static tvm::runtime::hexagon::HexagonRPCServer* g_hexagon_rpc_server = nullptr;

Review comment:
       Use an anonymous function to return the static global and init on first use

##########
File path: tests/python/contrib/test_hexagon/rpc/conftest.py
##########
@@ -0,0 +1,82 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import os
+
+import tvm.testing
+
+
+def pytest_addoption(parser):
+    parser.addoption(
+        "--serial-number",
+        required=True,
+        help=("Android device serial number list from 'adb' command."),
+    )
+
+
+@pytest.fixture
+def android_serial_number(request):
+    return request.config.getoption("--serial-number")
+
+
+@tvm.testing.fixture
+def tvm_tracker_host():
+    return os.environ["TVM_TRACKER_HOST"]
+
+
+@tvm.testing.fixture
+def tvm_tracker_port():
+    return int(os.environ["TVM_TRACKER_PORT"])
+
+
+def _compose(args, decs):
+    """Helper to apply multiple markers"""
+    if len(args) > 0:
+        f = args[0]
+        for d in reversed(decs):
+            f = d(f)
+        return f
+    return decs
+
+
+def requires_rpc_tracker(*args):
+    """Mark a test as requiring an RPC tracker to exist in
+    the host environment to run."""
+    _requires_rpc_tracker = [
+        *tvm.testing.requires_rpc(),
+        pytest.mark.skipif(
+            os.environ.get("TVM_TRACKER_HOST") == None,
+            reason="Missing environment variable, TVM_TRACKER_HOST",
+        ),
+        pytest.mark.skipif(
+            os.environ.get("TVM_TRACKER_PORT") == None,
+            reason="Missing environment variable, TVM_TRACKER_PORT",
+        ),
+    ]
+
+    return _compose(args, _requires_rpc_tracker)
+
+
+def requires_hexagon_toolchain(*args):
+    _requires_hexagon_toolchain = [
+        pytest.mark.skipif(
+            os.environ.get("HEXAGON_TOOLCHAIN") == None,
+            reason="HEXAGON_TOOLCHAIN environment variable is required to run this test.",
+        ),
+    ]

Review comment:
       These are defined also in `test_hexagon/proxy_rpc/` for the sake of common code can you put these in `test_hexagon/conftest.py`?

##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -240,11 +344,36 @@ if (USE_HEXAGON_DEVICE STREQUAL "${PICK_NONE}")
     file(GLOB RUNTIME_HEXAGON_SRCS src/runtime/hexagon/hexagon/*.cc)
   elseif(BUILD_FOR_ANDROID AND HEXAGON_SDK_PATH_DEFINED)
     list(APPEND RUNTIME_HEXAGON_SRCS src/runtime/hexagon/proxy_rpc/device_api.cc)
-  else()
+  elseif(USE_HEXAGON_RPC)

Review comment:
       See comment ⬆️   about breaking out not reusing the USE_HEXAGON_DEVICE block. It will also be easier to delete the USE_HEXAGON_DEVICE flow down the line if they are decoupled. 

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])

Review comment:
       We don't _need_ root do we? We may not always be able to root.

##########
File path: cmake/modules/Hexagon.cmake
##########
@@ -217,16 +317,20 @@ if(USE_HEXAGON_DEVICE STREQUAL "${PICK_SIM}")
       "-DHEXAGON_ARCH=${USE_HEXAGON_ARCH}"
     INSTALL_COMMAND "true"
   )
-elseif(USE_HEXAGON_DEVICE STREQUAL "${PICK_HW}")
+elseif((USE_HEXAGON_DEVICE STREQUAL "${PICK_HW}") OR (USE_HEXAGON_RPC AND BUILD_FOR_ANDROID))

Review comment:
       Can we decouple the `(USE_HEXAGON_DEVICE STREQUAL "${PICK_HW}")` and `(USE_HEXAGON_RPC AND BUILD_FOR_ANDROID)` cases into separate conditionals? It's a bit tricky to follow the flow and easy to make an error in the build for one or the other flows unintentionally when grouping them together like this.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);

Review comment:
       The names here are a bit confusing. Write calls SetReadBuffer, and below Read calls GetWriteBuffer.. I think I understand that HexagonRPCServer::Write sets the buffer to "read from" for the IO handler. 
   
   Maybe some docs or more intuitive naming could help demystify.

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+

Review comment:
       Can we add a check to the size of `self._workspace` and ensure its contents do not exceed some configurable size (e.g. a few GBs)? If it does, then we should delete the oldest entries according to the datetime directory name. 

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]
+            )
+
+        # Run server and connect to tracker
+        subprocess.Popen(
+            self._adb_device_sub_cmd + ["shell", f"cd {self._workspace} && ./android_bash.sh"],
+            stdout=subprocess.PIPE,
+            stdin=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+        )
+
+    def hexagon_setup(self):
+        """Upload Hexagon artifacts on Android."""
+        for item in HEXAGON_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            dst_path = f"{self._workspace}/{item}"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_path])
+
+    def hexagon_session_setup(self, remote_kw: dict):
+        """Setup Hexagon RPC Session from host to Hexagon device.
+
+        Parameters
+        ----------
+        remote_kw : dict
+            RPC tracker configs.
+        """
+        hexagon_remote_kw = dict(remote_kw)
+        hexagon_remote_kw["key"] = self.HEXAGON_REMOTE_DEVICE_KEY
+        self.session = Session(hexagon_remote_kw)
+
+    def get_module(self, module_name: str):
+        """Load a Hexagon TVM module, already uploaded on Android, on Hexagon and return the module.
+
+        Parameters
+        ----------
+        module_name : str
+            Module filename.
+
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        module_path = f"{self._workspace}/{module_name}"
+        self._mod = self.session.load_module(module_path)
+        return self._mod
+
+    def upload(self, host_path: Union[str, pathlib.Path], remote_filename: str):
+        """Upload a file to remote(Android).
+
+        Parameters
+        ----------
+        host_path : Union[str, pathlib.Path]
+            File path on host.
+
+        remote_filename : str
+            File name on remote(Android).
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        src_path = str(host_path)
+        dst_remote_path = f"{self._workspace}/{remote_filename}"
+        subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_remote_path])
+
+    def get_local_graph_executor(self, libmod, remote_libmod_filename: str):
+        """Create a GraphModule.
+
+        Parameters
+        ----------
+        libmod : tvm.runtime.Module

Review comment:
       Can we elaborate here that this is the library module for the remote hexagon runtime?

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":

Review comment:
       nit (take it or leave it): We make a roundtrip for the aarch64 compiled libtvm_runtime.so via a copy step in cmake to `libtvm_runtime_android.so` and back to `libtvm_runtime.so` when pushing to android. If someone looks at the libtvm_runtime.so on the adb side they might not know where this is coming from. I guess having hexagon_rpc/libtvm_runtime.so is sufficient given that the hexagon libtvm_runtime is statically linked in to the skel, so there isn't any ambiguity. My preference would be for either tvm_rpc to link against `libtvm_runtime_android.so` or for it to be copied in cmake as `hexagon/libtvm_runtime.so` so that in either case `tvm_rpc` is linking against what is found in the `hexagon_rpc/` build directory. 

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]

Review comment:
       I don't think we need to forward or reverse any ports. Can you confirm?

##########
File path: src/runtime/hexagon/rpc/hexagon_rpc.idl
##########
@@ -0,0 +1,28 @@
+/*
+ * 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 "remote.idl"
+#include "AEEStdDef.idl"
+
+typedef sequence<octet> buffer;
+
+interface hexagon_rpc : remote_handle64 {
+  AEEResult send(in buffer buf);
+  AEEResult receive(rout buffer buf, rout int64_t buf_written_size);
+};

Review comment:
       🔥 

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]
+            )
+
+        # Run server and connect to tracker
+        subprocess.Popen(
+            self._adb_device_sub_cmd + ["shell", f"cd {self._workspace} && ./android_bash.sh"],
+            stdout=subprocess.PIPE,
+            stdin=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+        )
+
+    def hexagon_setup(self):
+        """Upload Hexagon artifacts on Android."""
+        for item in HEXAGON_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            dst_path = f"{self._workspace}/{item}"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_path])
+
+    def hexagon_session_setup(self, remote_kw: dict):
+        """Setup Hexagon RPC Session from host to Hexagon device.
+
+        Parameters
+        ----------
+        remote_kw : dict
+            RPC tracker configs.
+        """
+        hexagon_remote_kw = dict(remote_kw)
+        hexagon_remote_kw["key"] = self.HEXAGON_REMOTE_DEVICE_KEY
+        self.session = Session(hexagon_remote_kw)
+
+    def get_module(self, module_name: str):
+        """Load a Hexagon TVM module, already uploaded on Android, on Hexagon and return the module.
+
+        Parameters
+        ----------
+        module_name : str
+            Module filename.
+
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        module_path = f"{self._workspace}/{module_name}"
+        self._mod = self.session.load_module(module_path)
+        return self._mod
+
+    def upload(self, host_path: Union[str, pathlib.Path], remote_filename: str):
+        """Upload a file to remote(Android).
+
+        Parameters
+        ----------
+        host_path : Union[str, pathlib.Path]
+            File path on host.
+
+        remote_filename : str
+            File name on remote(Android).
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        src_path = str(host_path)
+        dst_remote_path = f"{self._workspace}/{remote_filename}"
+        subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_remote_path])
+
+    def get_local_graph_executor(self, libmod, remote_libmod_filename: str):

Review comment:
       I was going to suggest `get_remote_graph_executor` instead. Though, pheraps `get_graph_executor` is better because the python GraphModule is local but the libmod that it calls into is remote, and then update the docstring to indicate as much.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {

Review comment:
       read_buffer_size_bytes_ can be uninitialized if SetReadBuffer is not called.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);
+    rpc_server_.ProcessOnePacket();
+    return (int64_t)data_len_bytes;
+  }
+
+  int64_t Read(uint8_t* buf, size_t read_len_bytes) {
+    return io_.GetWriteBuffer(buf, read_len_bytes);
+  }
+
+ private:
+  HexagonIOHandler io_;
+  MinRPCServer<HexagonIOHandler> rpc_server_;
+};
+
+}  // namespace hexagon
+}  // namespace runtime
+}  // namespace tvm
+
+static tvm::runtime::hexagon::HexagonRPCServer* g_hexagon_rpc_server = nullptr;
+
+static AEEResult hexagon_rpc_server_init() {
+  uint8_t* receive_buffer = new uint8_t[TVM_HEXAGON_RPC_BUFF_SIZE_BYTES];

Review comment:
       Nothing as far as I can tell is protecting against overflowing the receive_buffer TVM_HEXAGON_RPC_BUFF_SIZE_BYTES, we don't bass the receive_buffer_size to the iohandler (via the server).

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);

Review comment:
       Same for other FARFs
   

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}

Review comment:
       Constructor only partially initializes the object. It initializes read_buffer_ in construction, but is overwritten by SetReadBuffer which is mandatory to call before use because `read_buffer_size_bytes_` is uninitialized. Should be one or the other (fully uninitialized, or fully initialized).

##########
File path: src/runtime/hexagon/rpc/android/session.cc
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file hexagon_session.cc
+ */
+
+#include <tvm/runtime/registry.h>
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+}
+
+#include <tvm/runtime/logging.h>
+
+#include <string>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "../hexagon_rpc.h"
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonTransportChannel : public RPCChannel {
+ public:
+  explicit HexagonTransportChannel(const std::string& uri) {
+    if (_handle != AEE_EUNKNOWN) return;
+
+    enable_unsigned_pd(true);
+    set_remote_stack_size(128 * 1024);

Review comment:
       Let us make the stack size a parameter as well so we can configure it from python.




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: apps/cpp_rpc/CMakeLists.txt
##########
@@ -43,4 +45,12 @@ target_include_directories(
   PUBLIC DMLC_PATH
 )
 
-target_link_libraries(tvm_rpc tvm_runtime)
+if (BUILD_FOR_ANDROID AND USE_HEXAGON_SDK)
+  find_hexagon_sdk_root("${USE_HEXAGON_SDK}" "${USE_HEXAGON_ARCH}")
+  link_directories(${HEXAGON_REMOTE_ROOT})
+  list(APPEND TVM_RPC_LINKER_LIBS cdsprpc log)
+endif()

Review comment:
       You're absolutely right, we don't need to do that. For some reason when I link them in `Hexagon.cmake` file with exactly the same approach, link_directories fails to find the libraries. That's how I worked around this but we need to fix it before merging this.




-- 
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 #9631: [Hexagon] Add RPC Mechanism for Hexagon

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



##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]
+            )
+
+        # Run server and connect to tracker
+        subprocess.Popen(
+            self._adb_device_sub_cmd + ["shell", f"cd {self._workspace} && ./android_bash.sh"],
+            stdout=subprocess.PIPE,
+            stdin=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+        )
+
+    def hexagon_setup(self):
+        """Upload Hexagon artifacts on Android."""
+        for item in HEXAGON_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            dst_path = f"{self._workspace}/{item}"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_path])
+
+    def hexagon_session_setup(self, remote_kw: dict):
+        """Setup Hexagon RPC Session from host to Hexagon device.
+
+        Parameters
+        ----------
+        remote_kw : dict
+            RPC tracker configs.
+        """
+        hexagon_remote_kw = dict(remote_kw)
+        hexagon_remote_kw["key"] = self.HEXAGON_REMOTE_DEVICE_KEY
+        self.session = Session(hexagon_remote_kw)
+
+    def get_module(self, module_name: str):
+        """Load a Hexagon TVM module, already uploaded on Android, on Hexagon and return the module.
+
+        Parameters
+        ----------
+        module_name : str
+            Module filename.
+
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        module_path = f"{self._workspace}/{module_name}"
+        self._mod = self.session.load_module(module_path)
+        return self._mod
+
+    def upload(self, host_path: Union[str, pathlib.Path], remote_filename: str):
+        """Upload a file to remote(Android).
+
+        Parameters
+        ----------
+        host_path : Union[str, pathlib.Path]
+            File path on host.
+
+        remote_filename : str
+            File name on remote(Android).
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        src_path = str(host_path)
+        dst_remote_path = f"{self._workspace}/{remote_filename}"
+        subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_remote_path])
+
+    def get_local_graph_executor(self, libmod, remote_libmod_filename: str):

Review comment:
       fixed here: https://github.com/apache/tvm/pull/9631/commits/299a378d277bca48c1fc2ac2a5c9f79dd9052595

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])
+        # Removed pre-defined forward/reverse rules
+        subprocess.check_call(self._adb_device_sub_cmd + ["forward", "--remove-all"])
+        subprocess.check_call(self._adb_device_sub_cmd + ["reverse", "--remove-all"])
+
+        # Enable port reverse for RPC tracker
+        subprocess.check_call(
+            self._adb_device_sub_cmd
+            + ["reverse", f"tcp:{rpc_tracker_port}", f"tcp:{rpc_tracker_port}"]
+        )
+        # Enable port forward for RPC server. We forward 9 ports after the rpc_server_port.
+        for i in range(0, 10):
+            subprocess.check_call(
+                self._adb_device_sub_cmd
+                + ["forward", f"tcp:{rpc_server_port+i}", f"tcp:{rpc_server_port+i}"]
+            )
+
+        # Run server and connect to tracker
+        subprocess.Popen(
+            self._adb_device_sub_cmd + ["shell", f"cd {self._workspace} && ./android_bash.sh"],
+            stdout=subprocess.PIPE,
+            stdin=subprocess.PIPE,
+            stderr=subprocess.PIPE,
+        )
+
+    def hexagon_setup(self):
+        """Upload Hexagon artifacts on Android."""
+        for item in HEXAGON_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            dst_path = f"{self._workspace}/{item}"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_path])
+
+    def hexagon_session_setup(self, remote_kw: dict):
+        """Setup Hexagon RPC Session from host to Hexagon device.
+
+        Parameters
+        ----------
+        remote_kw : dict
+            RPC tracker configs.
+        """
+        hexagon_remote_kw = dict(remote_kw)
+        hexagon_remote_kw["key"] = self.HEXAGON_REMOTE_DEVICE_KEY
+        self.session = Session(hexagon_remote_kw)
+
+    def get_module(self, module_name: str):
+        """Load a Hexagon TVM module, already uploaded on Android, on Hexagon and return the module.
+
+        Parameters
+        ----------
+        module_name : str
+            Module filename.
+
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        module_path = f"{self._workspace}/{module_name}"
+        self._mod = self.session.load_module(module_path)
+        return self._mod
+
+    def upload(self, host_path: Union[str, pathlib.Path], remote_filename: str):
+        """Upload a file to remote(Android).
+
+        Parameters
+        ----------
+        host_path : Union[str, pathlib.Path]
+            File path on host.
+
+        remote_filename : str
+            File name on remote(Android).
+        Returns
+        -------
+        TVMModule :
+            A TVM Module loaded on hexagon.
+        """
+        src_path = str(host_path)
+        dst_remote_path = f"{self._workspace}/{remote_filename}"
+        subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, dst_remote_path])
+
+    def get_local_graph_executor(self, libmod, remote_libmod_filename: str):
+        """Create a GraphModule.
+
+        Parameters
+        ----------
+        libmod : tvm.runtime.Module

Review comment:
       added here: https://github.com/apache/tvm/pull/9631/commits/299a378d277bca48c1fc2ac2a5c9f79dd9052595

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":

Review comment:
       Thanks for the suggestions. I picked the second one here: https://github.com/apache/tvm/pull/9631/commits/b16c8b06dca8a83264dab0c7f7b4710f58f8ead9

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {

Review comment:
       I initialized it at the constructor as you suggested in previous comment.

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+
+        # Upload RPC server and libraries
+        subprocess.check_call(self._adb_device_sub_cmd + ["shell", "mkdir", "-p", self._workspace])
+
+        # create bash script
+        android_bash_script_path = get_hexagon_rpc_dir() / "android_bash.sh"
+        with open(get_hexagon_rpc_dir() / "android_bash.sh.template", "r") as src_f:
+            if os.path.exists(android_bash_script_path):
+                os.remove(android_bash_script_path)
+            with open(android_bash_script_path, "w") as dest_f:
+                for line in src_f.readlines():
+                    if "<RPC_TRACKER_HOST>" in line:
+                        line = line.replace("<RPC_TRACKER_HOST>", str(rpc_tracker_host))
+                    if "<RPC_TRACKER_PORT>" in line:
+                        line = line.replace("<RPC_TRACKER_PORT>", str(rpc_tracker_port))
+                    if "<HEXAGON_REMOTE_DEVICE_KEY>" in line:
+                        line = line.replace(
+                            "<HEXAGON_REMOTE_DEVICE_KEY>", self.HEXAGON_REMOTE_DEVICE_KEY
+                        )
+                    if "<RPC_SERVER_PORT>" in line:
+                        line = line.replace("<RPC_SERVER_PORT>", str(rpc_server_port))
+                    dest_f.write(line)
+
+        # make shell script executable
+        android_bash_stat = os.stat(android_bash_script_path)
+        os.chmod(android_bash_script_path, android_bash_stat.st_mode | stat.S_IEXEC)
+
+        # push files
+        for item in RPC_SERVER_FILES:
+            src_path = get_hexagon_rpc_dir() / item
+            destination = f"{self._workspace}/{item}"
+            if item == "libtvm_runtime_android.so":
+                destination = f"{self._workspace}/libtvm_runtime.so"
+            subprocess.check_call(self._adb_device_sub_cmd + ["push", src_path, destination])
+
+        # enable root for adb
+        subprocess.check_call(self._adb_device_sub_cmd + ["root"])

Review comment:
       That's right, we don't need it: https://github.com/apache/tvm/pull/9631/commits/076619f0d5a6cc91e3bc16882c2cd79743f40be9

##########
File path: python/tvm/contrib/hexagon/build.py
##########
@@ -0,0 +1,259 @@
+# 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 top-level glue functions for building Hexagon."""
+
+import pathlib
+import os
+import subprocess
+from typing import Union
+import stat
+import datetime
+
+import tvm
+from ..._ffi import libinfo
+from .session import Session
+
+
+RPC_SERVER_FILES = ["tvm_rpc_android", "libtvm_runtime_android.so", "android_bash.sh"]
+
+HEXAGON_FILES = ["libhexagon_rpc_skel.so"]
+
+HEXAGON_RPC_DIR = None
+
+ANDROID_HEXAGON_TEST_BASE_DIR = pathlib.Path("/data/local/tmp/hexagon_test")
+
+
+def get_hexagon_rpc_dir() -> pathlib.Path:
+    """Find the Hexagon library.
+
+    Returns
+    -------
+    str :
+        The path to the Hexagon library
+    """
+    global HEXAGON_RPC_DIR
+    if HEXAGON_RPC_DIR is None:
+        for path in libinfo.find_lib_path():
+            rpc_dir = os.path.join(os.path.dirname(path), "hexagon_rpc")
+            if os.path.isdir(rpc_dir):
+                HEXAGON_RPC_DIR = pathlib.Path(rpc_dir)
+                break
+        else:
+            raise "hexagon_rpc was not found."
+    return HEXAGON_RPC_DIR
+
+
+class HexagonLauncher:
+    """Hexagon Launcher"""
+
+    def __init__(self, serial_number: str):
+        """Configure a new HexagonLauncher
+
+        Parameters
+        ----------
+        serial_number : str
+            Android device serial number from android 'adb' command.
+        """
+        # Hexagon RPCSession
+        self.session = None
+
+        self._serial_number = serial_number
+        self._adb_device_sub_cmd = ["adb", "-s", self._serial_number]
+        self._mod = None
+        self._workspace = None
+
+    HEXAGON_REMOTE_DEVICE_KEY = "hexagon-dev"
+
+    def android_run_rpc(
+        self,
+        workspace_dir: Union[str, pathlib.Path] = None,
+        rpc_server_port: int = 7070,
+        rpc_tracker_host: str = "0.0.0.0",
+        rpc_tracker_port: int = 9190,
+    ):
+        """Upload Android artifacts and run RPC server on Android.
+
+        Parameters
+        ----------
+        workspace_dir : Union[str, pathlib.Path]
+            Workspace directory used on Android to upload artifacts.
+
+        rpc_server_port : int
+            Android RPC server port number
+
+        rpc_tracker_host : str
+            RPC tracker IP on host
+
+        rpc_tracker_port : int
+            RPC tracker port on host
+        """
+        if not workspace_dir:
+            self._workspace = str(
+                ANDROID_HEXAGON_TEST_BASE_DIR
+                / datetime.datetime.now().strftime("%Y-%m-%dT%H-%M-%S")
+            )
+        else:
+            self._workspace = workspace_dir
+

Review comment:
       Added in commit https://github.com/apache/tvm/pull/9631/commits/cea8757ddc39a4c9589e1d5394a4ae431beb1660  

##########
File path: src/runtime/hexagon/rpc/android/session.cc
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file hexagon_session.cc
+ */
+
+#include <tvm/runtime/registry.h>
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+}
+
+#include <tvm/runtime/logging.h>
+
+#include <string>
+
+#include "../../../rpc/rpc_channel.h"
+#include "../../../rpc/rpc_endpoint.h"
+#include "../../../rpc/rpc_session.h"
+#include "../hexagon_rpc.h"
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonTransportChannel : public RPCChannel {
+ public:
+  explicit HexagonTransportChannel(const std::string& uri) {
+    if (_handle != AEE_EUNKNOWN) return;
+
+    enable_unsigned_pd(true);
+    set_remote_stack_size(128 * 1024);

Review comment:
       done in: https://github.com/apache/tvm/pull/9631/commits/46562320069bb139e96bb86974b2a3cdbbc47c62

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);
+    size_t written_size = static_cast<size_t>(
+        write_buffer_.sputn(reinterpret_cast<const char*>(buf), write_len_bytes));
+    if (written_size != write_len_bytes) {
+      FARF(ALWAYS, "HexagonIOHandler written_size failed");
+    }
+    return (ssize_t)written_size;
+  }
+
+  void MessageDone() {}
+
+  ssize_t PosixRead(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixRead called, %d, %d", read_len_bytes,
+         read_buffer_size_bytes_);
+
+    uint32_t bytes_to_read = 0;
+    if ((read_buffer_size_bytes_ - read_len_bytes) < 0) {
+      bytes_to_read = read_buffer_size_bytes_;
+    } else {
+      bytes_to_read = read_len_bytes;
+    }
+
+    std::memcpy(buf, read_buffer_, bytes_to_read);
+    read_buffer_ += bytes_to_read;
+    read_buffer_size_bytes_ -= bytes_to_read;
+    if (bytes_to_read != read_len_bytes) {
+      FARF(ERROR, "Error bytes_to_read (%d) < read_len_bytes (%d).", bytes_to_read, read_len_bytes);
+    }
+    return (ssize_t)bytes_to_read;
+  }
+
+  void SetReadBuffer(const uint8_t* buf, size_t buf_size_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler SetReadBuffer called: %d, prev read_buffer_size_bytes_: ",
+         buf_size_bytes, read_buffer_size_bytes_);
+    read_buffer_ = buf;
+    read_buffer_size_bytes_ = buf_size_bytes;
+  }
+
+  int64_t GetWriteBuffer(uint8_t* buf, size_t read_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler GetWriteBuffer called, read_len_bytes: %d", read_len_bytes);
+    return write_buffer_.sgetn(reinterpret_cast<char*>(buf), read_len_bytes);
+  }
+
+  void Close() { FARF(ALWAYS, "HexagonIOHandler Close called"); }
+
+  void Exit(int code) { exit(code); }
+
+ private:
+  const uint8_t* read_buffer_;
+  uint32_t read_buffer_size_bytes_;
+
+  std::stringbuf write_buffer_;
+};
+
+class HexagonRPCServer {
+ public:
+  explicit HexagonRPCServer(uint8_t* receive_buffer) : io_{receive_buffer}, rpc_server_{&io_} {};
+
+  int64_t Write(const uint8_t* data, size_t data_len_bytes) {
+    io_.SetReadBuffer(data, data_len_bytes);

Review comment:
       agreed, it is really confusing. I added some documentation here: https://github.com/apache/tvm/pull/9631/commits/e57c7aafcc650a644e5d93d4ba7145725c6cd538
   
   Please let me know if you have more suggestions.

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {

Review comment:
       https://github.com/apache/tvm/pull/9631/commits/e57c7aafcc650a644e5d93d4ba7145725c6cd538

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}

Review comment:
       initialized the `read_buffer_size_bytes_` in https://github.com/apache/tvm/pull/9631/commits/e57c7aafcc650a644e5d93d4ba7145725c6cd538

##########
File path: python/tvm/contrib/hexagon/session.py
##########
@@ -0,0 +1,71 @@
+# 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 a Session class for Hexagon devices."""
+
+from tvm import rpc as _rpc
+
+
+class Session:
+    """Hexagon Device Session
+
+    Parameters
+    ----------
+    remote_kw : dict
+        Remote configs for RPC tracker.
+
+    session_name : str
+        Hexagon RPC session name.
+    """
+
+    def __init__(
+        self,
+        remote_kw: dict,
+        session_name: str = "hexagon-rpc",
+    ):
+        self._session_name = session_name
+        self._remote_kw = remote_kw
+        self._rpc = None
+        self.device = None
+
+    def __enter__(self):
+        if self.device:
+            return self

Review comment:
       else would create RPCSession and returns itself or raises exception. I think all cases are covered. Can you elaborate what's the confusion?

##########
File path: src/runtime/hexagon/rpc/hexagon/rpc_server.cc
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+extern "C" {
+#include <AEEStdDef.h>
+#include <AEEStdErr.h>
+#include <HAP_farf.h>
+#include <HAP_perf.h>
+#include <qurt_error.h>
+#include <qurt_hvx.h>
+}
+
+#include <dlfcn.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+#include "../../../library_module.h"
+#include "../../../minrpc/minrpc_server.h"
+#include "../../hexagon/hexagon_common.h"
+#include "hexagon_rpc.h"
+
+#define TVM_HEXAGON_RPC_BUFF_SIZE_BYTES 1024 * 1024
+
+#define TVM_LOG_CUSTOMIZE 1
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+
+class HexagonIOHandler {
+ public:
+  explicit HexagonIOHandler(uint8_t* read_buffer) : read_buffer_{read_buffer} {}
+
+  void MessageStart(size_t message_size_bytes) {}
+
+  ssize_t PosixWrite(const uint8_t* buf, size_t write_len_bytes) {
+    FARF(ALWAYS, "HexagonIOHandler PosixWrite called, write_len_bytes: %d", write_len_bytes);

Review comment:
       refactored in https://github.com/apache/tvm/pull/9631/commits/e57c7aafcc650a644e5d93d4ba7145725c6cd538

##########
File path: tests/python/contrib/test_hexagon/rpc/conftest.py
##########
@@ -0,0 +1,82 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import pytest
+import os
+
+import tvm.testing
+
+
+def pytest_addoption(parser):
+    parser.addoption(
+        "--serial-number",
+        required=True,
+        help=("Android device serial number list from 'adb' command."),
+    )
+
+
+@pytest.fixture
+def android_serial_number(request):
+    return request.config.getoption("--serial-number")
+
+
+@tvm.testing.fixture
+def tvm_tracker_host():
+    return os.environ["TVM_TRACKER_HOST"]
+
+
+@tvm.testing.fixture
+def tvm_tracker_port():
+    return int(os.environ["TVM_TRACKER_PORT"])
+
+
+def _compose(args, decs):
+    """Helper to apply multiple markers"""
+    if len(args) > 0:
+        f = args[0]
+        for d in reversed(decs):
+            f = d(f)
+        return f
+    return decs
+
+
+def requires_rpc_tracker(*args):
+    """Mark a test as requiring an RPC tracker to exist in
+    the host environment to run."""
+    _requires_rpc_tracker = [
+        *tvm.testing.requires_rpc(),
+        pytest.mark.skipif(
+            os.environ.get("TVM_TRACKER_HOST") == None,
+            reason="Missing environment variable, TVM_TRACKER_HOST",
+        ),
+        pytest.mark.skipif(
+            os.environ.get("TVM_TRACKER_PORT") == None,
+            reason="Missing environment variable, TVM_TRACKER_PORT",
+        ),
+    ]
+
+    return _compose(args, _requires_rpc_tracker)
+
+
+def requires_hexagon_toolchain(*args):
+    _requires_hexagon_toolchain = [
+        pytest.mark.skipif(
+            os.environ.get("HEXAGON_TOOLCHAIN") == None,
+            reason="HEXAGON_TOOLCHAIN environment variable is required to run this test.",
+        ),
+    ]

Review comment:
       Refactored conftest files here: https://github.com/apache/tvm/pull/9631/commits/aa61967796f975156bcf61030a441a53c909ecf4




-- 
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 pull request #9631: [Hexagon] Add RPC Mechanism for Hexagon

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


   @kparzysz-quic thanks for pointing out these issues. I will address these in the next PR for AOT executor.


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