You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2022/04/16 18:33:21 UTC

[GitHub] [tvm] kparzysz-quic commented on a diff in pull request #10967: [rpc] Implemented rpc logging

kparzysz-quic commented on code in PR #10967:
URL: https://github.com/apache/tvm/pull/10967#discussion_r851657346


##########
src/runtime/minrpc/minrpc_logger.cc:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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 "minrpc_logger.h"
+
+#include <string.h>
+#include <time.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/logging.h>
+
+#include <functional>
+#include <iostream>
+#include <sstream>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+void Logger::LogTVMValue(int tcode, TVMValue value) {
+  switch (tcode) {
+    case kDLInt:
+      this->LogVal<int64_t>("(int64)", value.v_int64);
+      break;
+    case kDLUInt:
+      this->LogVal<uint64_t>("(uint64)", value.v_int64);
+      break;
+    case kDLFloat: {
+      this->LogVal<float>("(float)", value.v_float64);
+      break;
+    }
+    case kTVMDataType: {
+      this->LogDLData("DLDataType(code,bits,lane)", &value.v_type);
+      break;
+    }
+    case kDLDevice: {
+      this->LogDLDevice("DLDevice(type,id)", &value.v_device);
+      break;
+    }
+    case kTVMPackedFuncHandle: {
+      this->LogVal<void*>("(PackedFuncHandle)", value.v_handle);
+      break;
+    }
+    case kTVMModuleHandle: {
+      this->LogVal<void*>("(ModuleHandle)", value.v_handle);
+      break;
+    }
+    case kTVMOpaqueHandle: {
+      this->LogVal<void*>("(OpaqueHandle)", value.v_handle);
+      break;
+    }
+    case kTVMDLTensorHandle: {
+      this->LogVal<void*>("(TensorHandle)", value.v_handle);
+      break;
+    }
+    case kTVMNDArrayHandle: {
+      this->LogVal<void*>("kTVMNDArrayHandle", value.v_handle);
+      break;
+    }
+    case kTVMNullptr:
+      this->LogString("Nullptr");
+      break;
+    case kTVMStr: {
+      this->LogString("\"");
+      this->LogString(value.v_str);
+      this->LogString("\"");
+      break;
+    }
+    case kTVMBytes: {
+      TVMByteArray* bytes = static_cast<TVMByteArray*>(value.v_handle);
+      int len = bytes->size;
+      this->LogVal<int64_t>("(Bytes) [size]: ", len);
+      if (PRINT_BYTES) {
+        this->LogString(", [Values]:");
+        this->LogString(" { ");
+        if (len > 0) {
+          this->LogVal<uint64_t>("", (uint8_t)bytes->data[0]);
+        }
+        for (int j = 1; j < len; j++) this->LogVal<uint64_t>(" - ", (uint8_t)bytes->data[j]);
+        this->LogString(" } ");
+      }
+      break;
+    }
+    default: {
+      this->LogString("ERROR-kUnknownTypeCode)");
+      break;
+    }
+  }
+  this->LogString("; ");
+}
+
+std::stringstream Logger::LogTime() {
+  char buf[100];
+  time_t t;
+  struct tm *timeptr, result, temp;
+  std::stringstream ss;
+
+  t = time(NULL);
+  timeptr = localtime_r(&t, &temp);
+  strftime(buf, sizeof(buf), "%a %m/%d/%Y %r", timeptr);
+
+  if (strptime(buf, "%a %m/%d/%Y %r", &result) == NULL) {
+    ss << "TIME UNKNOWN, ";
+  } else {
+    ss << result.tm_hour << ":" << result.tm_min << ":" << result.tm_sec << ", ";
+  }
+  return ss;
+}
+
+void Logger::OutputLog() {
+  LOG(INFO) << os_.str();
+  this->os_.str(std::string());
+}
+
+void MinRPCReturnsWithLog::ReturnVoid() {
+  next_->ReturnVoid();
+  logger_.LogString("-> ReturnVoid");
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnHandle(void* handle) {
+  next_->ReturnHandle(handle);
+  if (this->code_ == RPCCode::kGetGlobalFunc) {
+    this->registerHandleName(handle);
+  }
+  logger_.LogVal<void*>("-> ReturnHandle: ", handle);
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnException(const char* msg) {
+  next_->ReturnException(msg);
+  logger_.LogString("-> Exception: ");
+  logger_.LogString(msg);
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnPackedSeq(const TVMValue* arg_values, const int* type_codes,
+                                           int num_args) {
+  next_->ReturnPackedSeq(arg_values, type_codes, num_args);
+  processValues(&arg_values, &type_codes, &num_args);
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnCopyAck(uint64_t* num_bytes, uint8_t** data_ptr) {
+  next_->ReturnCopyAck(num_bytes, data_ptr);
+  logger_.LogVal<uint64_t>("-> CopyAck: ", *num_bytes);
+  logger_.LogVal<void*>(", ", reinterpret_cast<void*>(*data_ptr));
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnLastTVMError() {
+  const char* err = TVMGetLastError();
+  this->ReturnException(err);
+}
+
+void MinRPCReturnsWithLog::ThrowError(RPCServerStatus code, RPCCode info) {
+  next_->ThrowError(code, info);
+  logger_.LogString("-> ERROR");
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::processValues(const TVMValue** values, const int** tcodes,
+                                         int* num_args) {
+  if (*tcodes != nullptr) {
+    logger_.LogString("-> [");
+    for (int i = 0; i < *num_args; ++i) {
+      logger_.LogTVMValue((*tcodes)[i], (*values)[i]);
+
+      if ((*tcodes)[i] == kTVMOpaqueHandle) {
+        this->registerHandleName((*values)[i].v_handle);
+      }
+    }
+    logger_.LogString("]");
+  }
+}
+
+void MinRPCReturnsWithLog::resetCurrHandleName(RPCCode code) {
+  this->code_ = code;
+  logger_.LogString(RPCCodeToString(code));
+  logger_.LogString(", ");
+  this->CurrHandleName_.clear();
+}
+
+void MinRPCReturnsWithLog::updateCurrHandleName(const char* name) {
+  if (this->CurrHandleName_.length() != 0) {
+    this->CurrHandleName_.append("::");
+  }
+  this->CurrHandleName_.append(name);
+}
+
+void MinRPCReturnsWithLog::getHandleName(void* handle) {
+  if (array_tracker_.find(handle) != array_tracker_.end()) {
+    CurrHandleName_.append(array_tracker_[handle]);
+    logger_.LogHandleName(CurrHandleName_);
+  }
+}
+
+void MinRPCReturnsWithLog::releaseHandleName(void* handle) {
+  if (array_tracker_.find(handle) != array_tracker_.end()) {
+    logger_.LogHandleName(array_tracker_[handle]);
+    array_tracker_.erase(handle);
+  }
+}
+
+void MinRPCReturnsWithLog::registerHandleName(void* handle) {
+  const std::string newString = CurrHandleName_;
+  array_tracker_[handle] = newString;

Review Comment:
   ```
   array_tracker_[handle] = CurrHandleName_;
   ```



##########
src/runtime/minrpc/minrpc_server_logging.h:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+
+#ifndef TVM_RUNTIME_MINRPC_MINRPC_SERVER_LOGGING_H_
+#define TVM_RUNTIME_MINRPC_MINRPC_SERVER_LOGGING_H_
+
+#include "minrpc_logger.h"
+#include "minrpc_server.h"
+
+namespace tvm {
+namespace runtime {
+
+/*!
+ * \brief A minimum RPC server that logs the received commands.
+ *
+ * \tparam TIOHandler IO provider to provide io handling.
+ */
+template <typename TIOHandler>
+class MinRPCServerWithLog {
+ public:
+  explicit MinRPCServerWithLog(TIOHandler* io)
+      : ret_handler_(io),
+        ret_handler_WLog_(&ret_handler_),
+        exec_handler_(io, &ret_handler_WLog_),
+        exec_handler_WLog_(&exec_handler_),
+        next_(io, &exec_handler_WLog_) {}
+
+  bool ProcessOnePacket() { return next_.ProcessOnePacket(); }
+
+ private:
+  MinRPCReturns<TIOHandler> ret_handler_;
+  MinRPCExecute<TIOHandler> exec_handler_;
+  MinRPCReturnsWithLog ret_handler_WLog_;
+  MinRPCExecuteWithLog exec_handler_WLog_;
+  MinRPCServer<TIOHandler> next_;
+};
+
+/*!
+ * \brief A minimum RPC server that only logs the outgoing commands and received responses.
+ * (Does not process the packets or respond to them.)
+ *
+ * \tparam TIOHandler IO provider to provide io handling.
+ */
+template <typename TIOHandler>
+class MinRPCSniffer {
+ public:
+  explicit MinRPCSniffer(TIOHandler* io)
+      : io_(io),
+        ret_handler_(io_),
+        ret_handler_WLog_(&ret_handler_),
+        exec_handler_(&ret_handler_WLog_),
+        exec_handler_WLog_(&exec_handler_),
+        next_(io_, &exec_handler_WLog_) {}
+
+  bool ProcessOnePacket() { return next_.ProcessOnePacket(); }
+
+  void ProcessOneResponse() {
+    RPCCode code;
+    uint64_t packet_len = 0;
+
+    this->Read(&packet_len);
+    if (packet_len == 0) OutputLog();
+    this->Read(&code);
+    switch (code) {
+      case RPCCode::kReturn: {
+        HandleReturn();
+        break;
+      }
+      case RPCCode::kException: {
+        ret_handler_WLog_.ReturnException("");
+        break;
+      }
+      default: {
+        OutputLog();
+        break;
+      }
+    }
+  }
+
+ private:
+  void HandleReturn() {
+    int32_t num_args;
+    int32_t tcode;
+
+    this->Read(&num_args);
+    if (num_args == 1) {
+      this->Read(&tcode);
+      if (tcode == kTVMNullptr) {
+        ret_handler_WLog_.ReturnVoid();
+        return;
+      }
+      if (tcode == kTVMOpaqueHandle) {
+        uint64_t handle;
+        this->Read(&handle);
+        ret_handler_WLog_.ReturnHandle(reinterpret_cast<void*>(handle));
+        return;
+      }
+    }
+    OutputLog();
+  }
+
+  void OutputLog() { ret_handler_WLog_.getLogger()->OutputLog(); }
+
+  template <typename T>
+  void Read(T* data) {
+    static_assert(std::is_pod<T>::value, "need to be trival");

Review Comment:
   Please use `is_trivial && is_standard_layout` instead of `is_pod`.  `is_pod` is deprecated (in C++20 onwards).



##########
src/runtime/minrpc/minrpc_intrfc.h:
##########
@@ -0,0 +1,63 @@
+/*

Review Comment:
   Could you spell out the full name, i.e. make the file name `minrpc_interface.h`?



##########
src/runtime/minrpc/minrpc_logger.h:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.
+ */
+
+#ifndef TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+#define TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+
+#include <tvm/runtime/c_runtime_api.h>
+
+#include <functional>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+#define PRINT_BYTES false
+
+/*!
+ * \brief Generates a user readeable log on the console
+ */
+class Logger {
+ public:
+  Logger() {}
+
+  void LogString(const char* s) { os_ << s; }
+
+  void LogStr(std::string s) { os_ << s; }
+
+  template <typename T>
+  void LogVal(const char* s, T val) {
+    os_ << s << val;
+  }
+
+  void LogDLDevice(const char* s, DLDevice* dev) {
+    os_ << s << "(" << dev->device_type << "," << dev->device_id << ")";
+  }
+
+  void LogDLData(const char* s, DLDataType* data) {
+    os_ << s << "(" << (uint16_t)data->code << "," << (uint16_t)data->bits << "," << data->lanes
+        << ")";
+  }
+
+  void LogHandleName(std::string name) {
+    if (name.length() > 0) {
+      os_ << " <" << name.c_str() << ">";
+    }
+  }
+
+  std::stringstream LogTime();
+  void LogTVMValue(int tcode, TVMValue value);
+  void OutputLog();
+
+ private:
+  std::stringstream os_;
+};
+
+/*!
+ * \brief A wrapper for a MinRPCReturns object, that also logs the responses.
+ *
+ * \tparam ReturnInterface* underlying MinRPCReturns that generates the responses.
+ */
+class MinRPCReturnsWithLog : public ReturnInterface {
+ public:
+  /*!
+   * \brief Constructor.
+   * \param io The IO handler.
+   */
+  explicit MinRPCReturnsWithLog(ReturnInterface* next) : next_(next), logger_() {}
+
+  ~MinRPCReturnsWithLog() {}
+
+  void ReturnVoid();
+
+  void ReturnHandle(void* handle);
+
+  void ReturnException(const char* msg);
+
+  void ReturnPackedSeq(const TVMValue* arg_values, const int* type_codes, int num_args);
+
+  void ReturnCopyAck(uint64_t* num_bytes, uint8_t** data_ptr);
+
+  void ReturnLastTVMError();
+
+  void ThrowError(RPCServerStatus code, RPCCode info = RPCCode::kNone);
+
+  void processValues(const TVMValue** values, const int** tcodes, int* num_args);
+
+  void resetCurrHandleName(RPCCode code);
+
+  void updateCurrHandleName(const char* name);
+
+  void getHandleName(void* handle);
+
+  void releaseHandleName(void* handle);
+
+  Logger* getLogger() { return &logger_; }
+
+ private:
+  void registerHandleName(void* handle);
+
+  ReturnInterface* next_;
+  std::string CurrHandleName_;
+  std::unordered_map<void*, std::string> array_tracker_;
+  RPCCode code_;
+  Logger logger_;
+};
+
+/*!
+ * \brief A wrapper for a MinRPCExecute object, that also logs the responses.
+ *
+ * \tparam RExecInterface* underlying MinRPCExecute that processes the packets.
+ */
+class MinRPCExecuteWithLog : public ExecInterface {
+ public:
+  explicit MinRPCExecuteWithLog(ExecInterface* next) : next_(next) {
+    this->ret_handler_ = reinterpret_cast<MinRPCReturnsWithLog*>(next_->getReturnInterface());
+    this->logger_ = ret_handler_->getLogger();
+  }
+
+  ~MinRPCExecuteWithLog() {}
+
+  void ExecInitServer(int* _num_args);

Review Comment:
   No names with leading `_`, please.



##########
src/runtime/minrpc/minrpc_logger.h:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.
+ */
+
+#ifndef TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+#define TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+
+#include <tvm/runtime/c_runtime_api.h>
+
+#include <functional>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+#define PRINT_BYTES false
+
+/*!
+ * \brief Generates a user readeable log on the console
+ */
+class Logger {
+ public:
+  Logger() {}
+
+  void LogString(const char* s) { os_ << s; }
+
+  void LogStr(std::string s) { os_ << s; }
+
+  template <typename T>
+  void LogVal(const char* s, T val) {
+    os_ << s << val;
+  }
+
+  void LogDLDevice(const char* s, DLDevice* dev) {
+    os_ << s << "(" << dev->device_type << "," << dev->device_id << ")";
+  }
+
+  void LogDLData(const char* s, DLDataType* data) {
+    os_ << s << "(" << (uint16_t)data->code << "," << (uint16_t)data->bits << "," << data->lanes
+        << ")";
+  }
+
+  void LogHandleName(std::string name) {
+    if (name.length() > 0) {
+      os_ << " <" << name.c_str() << ">";
+    }
+  }
+
+  std::stringstream LogTime();
+  void LogTVMValue(int tcode, TVMValue value);
+  void OutputLog();
+
+ private:
+  std::stringstream os_;
+};
+
+/*!
+ * \brief A wrapper for a MinRPCReturns object, that also logs the responses.
+ *
+ * \tparam ReturnInterface* underlying MinRPCReturns that generates the responses.
+ */
+class MinRPCReturnsWithLog : public ReturnInterface {
+ public:
+  /*!
+   * \brief Constructor.
+   * \param io The IO handler.
+   */
+  explicit MinRPCReturnsWithLog(ReturnInterface* next) : next_(next), logger_() {}
+
+  ~MinRPCReturnsWithLog() {}
+
+  void ReturnVoid();
+
+  void ReturnHandle(void* handle);
+
+  void ReturnException(const char* msg);
+
+  void ReturnPackedSeq(const TVMValue* arg_values, const int* type_codes, int num_args);
+
+  void ReturnCopyAck(uint64_t* num_bytes, uint8_t** data_ptr);
+
+  void ReturnLastTVMError();
+
+  void ThrowError(RPCServerStatus code, RPCCode info = RPCCode::kNone);
+
+  void processValues(const TVMValue** values, const int** tcodes, int* num_args);
+
+  void resetCurrHandleName(RPCCode code);
+
+  void updateCurrHandleName(const char* name);
+
+  void getHandleName(void* handle);
+
+  void releaseHandleName(void* handle);
+
+  Logger* getLogger() { return &logger_; }

Review Comment:
   Function names should start with capital letters (TVM uses Google's naming conventions).



##########
src/runtime/minrpc/minrpc_logger.cc:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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 "minrpc_logger.h"
+
+#include <string.h>
+#include <time.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/logging.h>
+
+#include <functional>
+#include <iostream>
+#include <sstream>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+void Logger::LogTVMValue(int tcode, TVMValue value) {
+  switch (tcode) {
+    case kDLInt:
+      this->LogVal<int64_t>("(int64)", value.v_int64);
+      break;
+    case kDLUInt:
+      this->LogVal<uint64_t>("(uint64)", value.v_int64);
+      break;
+    case kDLFloat: {
+      this->LogVal<float>("(float)", value.v_float64);
+      break;
+    }
+    case kTVMDataType: {
+      this->LogDLData("DLDataType(code,bits,lane)", &value.v_type);
+      break;
+    }
+    case kDLDevice: {
+      this->LogDLDevice("DLDevice(type,id)", &value.v_device);
+      break;
+    }
+    case kTVMPackedFuncHandle: {
+      this->LogVal<void*>("(PackedFuncHandle)", value.v_handle);
+      break;
+    }
+    case kTVMModuleHandle: {
+      this->LogVal<void*>("(ModuleHandle)", value.v_handle);
+      break;
+    }
+    case kTVMOpaqueHandle: {
+      this->LogVal<void*>("(OpaqueHandle)", value.v_handle);
+      break;
+    }
+    case kTVMDLTensorHandle: {
+      this->LogVal<void*>("(TensorHandle)", value.v_handle);
+      break;
+    }
+    case kTVMNDArrayHandle: {
+      this->LogVal<void*>("kTVMNDArrayHandle", value.v_handle);
+      break;
+    }
+    case kTVMNullptr:
+      this->LogString("Nullptr");
+      break;
+    case kTVMStr: {
+      this->LogString("\"");
+      this->LogString(value.v_str);
+      this->LogString("\"");
+      break;
+    }
+    case kTVMBytes: {
+      TVMByteArray* bytes = static_cast<TVMByteArray*>(value.v_handle);
+      int len = bytes->size;
+      this->LogVal<int64_t>("(Bytes) [size]: ", len);
+      if (PRINT_BYTES) {
+        this->LogString(", [Values]:");
+        this->LogString(" { ");
+        if (len > 0) {
+          this->LogVal<uint64_t>("", (uint8_t)bytes->data[0]);
+        }
+        for (int j = 1; j < len; j++) this->LogVal<uint64_t>(" - ", (uint8_t)bytes->data[j]);
+        this->LogString(" } ");
+      }
+      break;
+    }
+    default: {
+      this->LogString("ERROR-kUnknownTypeCode)");
+      break;
+    }
+  }
+  this->LogString("; ");
+}
+
+std::stringstream Logger::LogTime() {
+  char buf[100];
+  time_t t;
+  struct tm *timeptr, result, temp;
+  std::stringstream ss;
+
+  t = time(NULL);
+  timeptr = localtime_r(&t, &temp);
+  strftime(buf, sizeof(buf), "%a %m/%d/%Y %r", timeptr);
+
+  if (strptime(buf, "%a %m/%d/%Y %r", &result) == NULL) {

Review Comment:
   Please use `std::put_time` and `std::localtime` instead to get formatted time string.



##########
src/runtime/minrpc/minrpc_logger.h:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.
+ */
+
+#ifndef TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+#define TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+
+#include <tvm/runtime/c_runtime_api.h>
+
+#include <functional>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+#define PRINT_BYTES false
+
+/*!
+ * \brief Generates a user readeable log on the console
+ */
+class Logger {
+ public:
+  Logger() {}
+
+  void LogString(const char* s) { os_ << s; }
+
+  void LogStr(std::string s) { os_ << s; }
+
+  template <typename T>
+  void LogVal(const char* s, T val) {
+    os_ << s << val;
+  }
+
+  void LogDLDevice(const char* s, DLDevice* dev) {
+    os_ << s << "(" << dev->device_type << "," << dev->device_id << ")";
+  }
+
+  void LogDLData(const char* s, DLDataType* data) {
+    os_ << s << "(" << (uint16_t)data->code << "," << (uint16_t)data->bits << "," << data->lanes
+        << ")";
+  }
+
+  void LogHandleName(std::string name) {
+    if (name.length() > 0) {
+      os_ << " <" << name.c_str() << ">";
+    }
+  }
+
+  std::stringstream LogTime();
+  void LogTVMValue(int tcode, TVMValue value);
+  void OutputLog();
+
+ private:
+  std::stringstream os_;
+};
+
+/*!
+ * \brief A wrapper for a MinRPCReturns object, that also logs the responses.
+ *
+ * \tparam ReturnInterface* underlying MinRPCReturns that generates the responses.
+ */
+class MinRPCReturnsWithLog : public ReturnInterface {
+ public:
+  /*!
+   * \brief Constructor.
+   * \param io The IO handler.
+   */
+  explicit MinRPCReturnsWithLog(ReturnInterface* next) : next_(next), logger_() {}
+
+  ~MinRPCReturnsWithLog() {}
+
+  void ReturnVoid();
+
+  void ReturnHandle(void* handle);
+
+  void ReturnException(const char* msg);
+
+  void ReturnPackedSeq(const TVMValue* arg_values, const int* type_codes, int num_args);
+
+  void ReturnCopyAck(uint64_t* num_bytes, uint8_t** data_ptr);
+
+  void ReturnLastTVMError();
+
+  void ThrowError(RPCServerStatus code, RPCCode info = RPCCode::kNone);
+
+  void processValues(const TVMValue** values, const int** tcodes, int* num_args);
+
+  void resetCurrHandleName(RPCCode code);
+
+  void updateCurrHandleName(const char* name);
+
+  void getHandleName(void* handle);
+
+  void releaseHandleName(void* handle);
+
+  Logger* getLogger() { return &logger_; }
+
+ private:
+  void registerHandleName(void* handle);

Review Comment:
   Same here.



##########
src/runtime/minrpc/minrpc_logger.cc:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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 "minrpc_logger.h"
+
+#include <string.h>
+#include <time.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/logging.h>
+
+#include <functional>
+#include <iostream>
+#include <sstream>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+void Logger::LogTVMValue(int tcode, TVMValue value) {
+  switch (tcode) {
+    case kDLInt:
+      this->LogVal<int64_t>("(int64)", value.v_int64);
+      break;
+    case kDLUInt:
+      this->LogVal<uint64_t>("(uint64)", value.v_int64);
+      break;
+    case kDLFloat: {
+      this->LogVal<float>("(float)", value.v_float64);
+      break;
+    }
+    case kTVMDataType: {
+      this->LogDLData("DLDataType(code,bits,lane)", &value.v_type);
+      break;
+    }
+    case kDLDevice: {
+      this->LogDLDevice("DLDevice(type,id)", &value.v_device);
+      break;
+    }
+    case kTVMPackedFuncHandle: {
+      this->LogVal<void*>("(PackedFuncHandle)", value.v_handle);
+      break;
+    }
+    case kTVMModuleHandle: {
+      this->LogVal<void*>("(ModuleHandle)", value.v_handle);
+      break;
+    }
+    case kTVMOpaqueHandle: {
+      this->LogVal<void*>("(OpaqueHandle)", value.v_handle);
+      break;
+    }
+    case kTVMDLTensorHandle: {
+      this->LogVal<void*>("(TensorHandle)", value.v_handle);
+      break;
+    }
+    case kTVMNDArrayHandle: {
+      this->LogVal<void*>("kTVMNDArrayHandle", value.v_handle);
+      break;
+    }
+    case kTVMNullptr:
+      this->LogString("Nullptr");
+      break;
+    case kTVMStr: {
+      this->LogString("\"");
+      this->LogString(value.v_str);
+      this->LogString("\"");
+      break;
+    }
+    case kTVMBytes: {
+      TVMByteArray* bytes = static_cast<TVMByteArray*>(value.v_handle);
+      int len = bytes->size;
+      this->LogVal<int64_t>("(Bytes) [size]: ", len);
+      if (PRINT_BYTES) {
+        this->LogString(", [Values]:");
+        this->LogString(" { ");
+        if (len > 0) {
+          this->LogVal<uint64_t>("", (uint8_t)bytes->data[0]);
+        }
+        for (int j = 1; j < len; j++) this->LogVal<uint64_t>(" - ", (uint8_t)bytes->data[j]);
+        this->LogString(" } ");
+      }
+      break;
+    }
+    default: {
+      this->LogString("ERROR-kUnknownTypeCode)");
+      break;
+    }
+  }
+  this->LogString("; ");
+}
+
+std::stringstream Logger::LogTime() {
+  char buf[100];
+  time_t t;
+  struct tm *timeptr, result, temp;
+  std::stringstream ss;
+
+  t = time(NULL);
+  timeptr = localtime_r(&t, &temp);
+  strftime(buf, sizeof(buf), "%a %m/%d/%Y %r", timeptr);
+
+  if (strptime(buf, "%a %m/%d/%Y %r", &result) == NULL) {
+    ss << "TIME UNKNOWN, ";
+  } else {
+    ss << result.tm_hour << ":" << result.tm_min << ":" << result.tm_sec << ", ";
+  }
+  return ss;
+}
+
+void Logger::OutputLog() {
+  LOG(INFO) << os_.str();
+  this->os_.str(std::string());
+}
+
+void MinRPCReturnsWithLog::ReturnVoid() {
+  next_->ReturnVoid();
+  logger_.LogString("-> ReturnVoid");
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnHandle(void* handle) {
+  next_->ReturnHandle(handle);
+  if (this->code_ == RPCCode::kGetGlobalFunc) {
+    this->registerHandleName(handle);
+  }
+  logger_.LogVal<void*>("-> ReturnHandle: ", handle);
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnException(const char* msg) {
+  next_->ReturnException(msg);
+  logger_.LogString("-> Exception: ");
+  logger_.LogString(msg);
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnPackedSeq(const TVMValue* arg_values, const int* type_codes,
+                                           int num_args) {
+  next_->ReturnPackedSeq(arg_values, type_codes, num_args);
+  processValues(&arg_values, &type_codes, &num_args);
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnCopyAck(uint64_t* num_bytes, uint8_t** data_ptr) {
+  next_->ReturnCopyAck(num_bytes, data_ptr);
+  logger_.LogVal<uint64_t>("-> CopyAck: ", *num_bytes);
+  logger_.LogVal<void*>(", ", reinterpret_cast<void*>(*data_ptr));
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::ReturnLastTVMError() {
+  const char* err = TVMGetLastError();
+  this->ReturnException(err);
+}
+
+void MinRPCReturnsWithLog::ThrowError(RPCServerStatus code, RPCCode info) {
+  next_->ThrowError(code, info);
+  logger_.LogString("-> ERROR");
+  logger_.OutputLog();
+}
+
+void MinRPCReturnsWithLog::processValues(const TVMValue** values, const int** tcodes,
+                                         int* num_args) {
+  if (*tcodes != nullptr) {
+    logger_.LogString("-> [");
+    for (int i = 0; i < *num_args; ++i) {
+      logger_.LogTVMValue((*tcodes)[i], (*values)[i]);
+
+      if ((*tcodes)[i] == kTVMOpaqueHandle) {
+        this->registerHandleName((*values)[i].v_handle);
+      }
+    }
+    logger_.LogString("]");
+  }
+}
+
+void MinRPCReturnsWithLog::resetCurrHandleName(RPCCode code) {
+  this->code_ = code;
+  logger_.LogString(RPCCodeToString(code));
+  logger_.LogString(", ");
+  this->CurrHandleName_.clear();
+}
+
+void MinRPCReturnsWithLog::updateCurrHandleName(const char* name) {
+  if (this->CurrHandleName_.length() != 0) {
+    this->CurrHandleName_.append("::");
+  }
+  this->CurrHandleName_.append(name);
+}
+
+void MinRPCReturnsWithLog::getHandleName(void* handle) {
+  if (array_tracker_.find(handle) != array_tracker_.end()) {
+    CurrHandleName_.append(array_tracker_[handle]);
+    logger_.LogHandleName(CurrHandleName_);
+  }
+}
+
+void MinRPCReturnsWithLog::releaseHandleName(void* handle) {
+  if (array_tracker_.find(handle) != array_tracker_.end()) {
+    logger_.LogHandleName(array_tracker_[handle]);
+    array_tracker_.erase(handle);
+  }
+}
+
+void MinRPCReturnsWithLog::registerHandleName(void* handle) {
+  const std::string newString = CurrHandleName_;
+  array_tracker_[handle] = newString;
+}
+
+void MinRPCExecuteWithLog::ExecInitServer(int* _num_args) {
+  setRPCCode(RPCCode::kInitServer);
+  logger_->LogString("Init Server");
+  next_->ExecInitServer(_num_args);
+}
+
+void MinRPCExecuteWithLog::ExecNormalCallFunc(uint64_t* call_handle, TVMValue** values,
+                                              int** tcodes, int* num_args) {
+  setRPCCode(RPCCode::kCallFunc);
+  logger_->LogVal<void*>("call_handle: ", reinterpret_cast<void*>(*call_handle));
+  ret_handler_->getHandleName(reinterpret_cast<void*>(*call_handle));
+  if (*num_args > 0) {
+    logger_->LogString(", ");
+  }
+  this->processValues(values, tcodes, num_args);
+  next_->ExecNormalCallFunc(call_handle, values, tcodes, num_args);
+}
+
+void MinRPCExecuteWithLog::ExecCopyFromRemote(DLTensor** arr, uint64_t* num_bytes,
+                                              uint8_t** temp_data) {
+  setRPCCode(RPCCode::kCopyFromRemote);
+  logger_->LogVal<void*>("data_handle: ", reinterpret_cast<void*>((*arr)->data));
+  logger_->LogDLDevice(", DLDevice(type,id):", &(*arr)->device);
+  logger_->LogVal<int64_t>(", ndim: ", (*arr)->ndim);
+  logger_->LogDLData(", DLDataType(code,bits,lane): ", &(*arr)->dtype);
+  logger_->LogVal<uint64_t>(", num_bytes:", *num_bytes);
+  next_->ExecCopyFromRemote(arr, num_bytes, temp_data);
+}
+
+int MinRPCExecuteWithLog::ExecCopyToRemote(DLTensor** arr, uint64_t* _num_bytes,
+                                           uint8_t** _data_ptr) {

Review Comment:
   Please don't use names that start with `_`.



##########
src/runtime/minrpc/minrpc_logger.cc:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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 "minrpc_logger.h"
+
+#include <string.h>
+#include <time.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/logging.h>
+
+#include <functional>
+#include <iostream>
+#include <sstream>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+void Logger::LogTVMValue(int tcode, TVMValue value) {
+  switch (tcode) {
+    case kDLInt:
+      this->LogVal<int64_t>("(int64)", value.v_int64);

Review Comment:
   Please remove the `this->` from everywhere in this PR.  In a vast majority of cases it's unnecessary and only adds to the clutter.



##########
src/runtime/minrpc/minrpc_logger.h:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.
+ */
+
+#ifndef TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+#define TVM_RUNTIME_MINRPC_MINRPC_LOGGER_H_
+
+#include <tvm/runtime/c_runtime_api.h>
+
+#include <functional>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+
+#include "minrpc_intrfc.h"
+#include "rpc_reference.h"
+
+namespace tvm {
+namespace runtime {
+
+#define PRINT_BYTES false
+
+/*!
+ * \brief Generates a user readeable log on the console
+ */
+class Logger {
+ public:
+  Logger() {}
+
+  void LogString(const char* s) { os_ << s; }
+
+  void LogStr(std::string s) { os_ << s; }
+
+  template <typename T>
+  void LogVal(const char* s, T val) {
+    os_ << s << val;
+  }
+
+  void LogDLDevice(const char* s, DLDevice* dev) {
+    os_ << s << "(" << dev->device_type << "," << dev->device_id << ")";
+  }
+
+  void LogDLData(const char* s, DLDataType* data) {
+    os_ << s << "(" << (uint16_t)data->code << "," << (uint16_t)data->bits << "," << data->lanes
+        << ")";
+  }
+
+  void LogHandleName(std::string name) {
+    if (name.length() > 0) {
+      os_ << " <" << name.c_str() << ">";
+    }
+  }
+
+  std::stringstream LogTime();
+  void LogTVMValue(int tcode, TVMValue value);
+  void OutputLog();
+
+ private:
+  std::stringstream os_;
+};
+
+/*!
+ * \brief A wrapper for a MinRPCReturns object, that also logs the responses.
+ *
+ * \tparam ReturnInterface* underlying MinRPCReturns that generates the responses.
+ */
+class MinRPCReturnsWithLog : public ReturnInterface {
+ public:
+  /*!
+   * \brief Constructor.
+   * \param io The IO handler.
+   */
+  explicit MinRPCReturnsWithLog(ReturnInterface* next) : next_(next), logger_() {}
+
+  ~MinRPCReturnsWithLog() {}
+
+  void ReturnVoid();
+
+  void ReturnHandle(void* handle);
+
+  void ReturnException(const char* msg);
+
+  void ReturnPackedSeq(const TVMValue* arg_values, const int* type_codes, int num_args);
+
+  void ReturnCopyAck(uint64_t* num_bytes, uint8_t** data_ptr);
+
+  void ReturnLastTVMError();
+
+  void ThrowError(RPCServerStatus code, RPCCode info = RPCCode::kNone);
+
+  void processValues(const TVMValue** values, const int** tcodes, int* num_args);
+
+  void resetCurrHandleName(RPCCode code);
+
+  void updateCurrHandleName(const char* name);
+
+  void getHandleName(void* handle);
+
+  void releaseHandleName(void* handle);
+
+  Logger* getLogger() { return &logger_; }
+
+ private:
+  void registerHandleName(void* handle);
+
+  ReturnInterface* next_;
+  std::string CurrHandleName_;

Review Comment:
   Use snake_case for variable names (i.e. curr_handle_name_).  At least be consistent within the same region of code.



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