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/04/25 21:12:53 UTC

[GitHub] [tvm] tqchen opened a new pull request #7919: [RUNTIME] Improve signal handling in python env.

tqchen opened a new pull request #7919:
URL: https://github.com/apache/tvm/pull/7919


   Python execution environment handles the signal by caching
   the signal a state and invokes the handler when execution
   goes into the python interpreter.
   
   This model can cause problem when runnning a long running
   c++ function. As keyboard interrupt can only be caught in the end.
   
   Additionally, because python registered special signal handlers.
   Socket operations can return EINTR that needs to be explicitly
   retried when the interrupt is not a KeyboardInterrupt.
   
   This PR adds the following changes to resolve these problems.
   
   - Allow execution env(python) to register CheckSignals function
     to the TVM runtime.
   - Add runtime::EnvCheckSignals to check the signal error.
   - Add retry when EINTR is encountered in socket.
   - Register the python C API functions in cython mode.
   
   To testout the EnvCheckSignals, run the following code
   
   ```python
   import tvm.testing
   tvm.testing.run_check_signal(10)
   ```
   
   Note that the C API functions are only registered in cython FFI mode
   because ctypes have problems invoking these functions. This however
   won't affect the correctness, but will defer the interrupt handling
   to function return sites.
   
   Co-authored-by: Andrew Reusch <ar...@octoml.ai>
   Co-authored-by: Robert Kimball <bo...@gmail.com>
   


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

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



[GitHub] [tvm] tqchen edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-830676898


   @areusch I pushed an alternative implementation that can preserve the python exception types.  I started  with this one but then decided to go with the approach because most exceptions are indeed KeyboardInterrupt. After the discussion I agree it can be more important to preserve the exception type.
   
   This implementation does that. The approach is similar to pybind's approach of handling the exception. When error happens a special exception is thrown, and the error is not cleared but propagated into the FFI layer through a specific return code.
   
   The only limitation here is that the stacktrace will only start from the FFI(the C++ traces are not tracked), but it is fine as the signal handling itself is also already async.
   
   I have also considered your proposal of having a SignalChecking function in the FFI layer and started with that impl before sending the PR. My conclusion so far there are only limited set that it is  cleaner to bring the API directly in the runtime. As the implementation hides behind a common API (registerCAPI) that won't change, we can consider refactor later. But given the processing code is usually short, it does not hurt to put them close to the implementation site either.


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

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



[GitHub] [tvm] junrushao1994 commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   Can we check if the address of ctypes.pythonapi.PyErr_CheckSignals is identical to what’s provided in Python.h? It should be a pretty straightforward check with a Python-C extension (no need to compile with TVM)


-- 
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] tqchen commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   i attempted the ctypes python api approach before and cannot make it work(caused segfault). Likely related to problems of ctypes wrapping or other issues


-- 
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] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513






-- 
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] junrushao1994 commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @areusch There are usecases in AutoTIR where we want to call back to python's stable apis in a multi-threaded C++ environment. It's not a strict dependency, and actually we can completely avoid doing this, but I'm just so curious, so TQ points me to this PR.
   
   After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] areusch commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @junrushao1994 just curious what the motivation is behind your question?


-- 
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] tqchen commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   cc @rkimball @areusch 
   
   cc @junrushao1994 as this might be relevant to long running aut-scheduler workloads 


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

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



[GitHub] [tvm] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,76 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {

Review comment:
       it is also used in the cython




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,89 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  /*!
+   * \brief Clear the error indicator.
+   */
+  typedef void (*F_PyErr_Clear)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+  /*!
+   * \brief PyErrClear function
+   */
+  F_PyErr_Clear pyerr_clear = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {
+      Update(symbol_name, &pyerr_check_signals, fptr);
+    } else if (symbol_name == "PyErr_Clear") {
+      Update(symbol_name, &pyerr_clear, fptr);
+    } else {
+      LOG(FATAL) << "Unknown env API " << symbol_name;
+    }
+  }
+
+  // implementation of tvm::runtime::EnvCheckSignals
+  void CheckSignals() {
+    // check python signal to see if there are exception raised
+    if (pyerr_check_signals != nullptr && (*pyerr_check_signals)() != 0) {
+      ICHECK_NOTNULL(pyerr_clear);
+      // clear the error so we can throw a new error
+      (*pyerr_clear)();
+      // Raise the error
+      LOG(FATAL) << "KeyboardInterrupt: Signal received";

Review comment:
       I think KeyboardInterrupt is consistent with what python is doing and user recognize this better. The error message already communicates the information




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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513


   @areusch There are usecases in AutoTIR where we want to call back to python in a multi-threaded C++ environment. We don't want to include `Python.h` because strictly speaking TVM is not designed to depend on python.
   
   Actually It's not a strict dependency, and we can completely avoid doing this in AutoTIR, but I'm just so curious, so TQ points me to this PR.
   
   After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513


   @areusch There are usecases in AutoTIR where we want to call back to python in a multi-threaded C++ environment (with GIL). However, C++ threads need to be properly initialized by calling [several python's stable APIs](https://docs.python.org/3/c-api/init.html#non-python-created-threads), but in fact we don't want to include `Python.h` because strictly speaking TVM is not designed to depend on python.
   
   Actually It's not a strict dependency, and we can completely avoid doing this in AutoTIR, but I'm just so curious, so TQ points me to this PR. After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] tqchen commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   cc @rkimball @areusch 
   
   cc @junrushao1994 as this might be relevant to long running aut-scheduler workloads 


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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/c_backend_api.h
##########
@@ -99,6 +100,19 @@ TVM_DLL void* TVMBackendAllocWorkspace(int device_type, int device_id, uint64_t
  */
 TVM_DLL int TVMBackendFreeWorkspace(int device_type, int device_id, void* ptr);
 
+/*!
+ * \brief Backend function to register execution environment(e.g. python)
+ *        specific C APIs.
+ *
+ * \note  We only register the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). In most cases we should use the PackedFunc FFI.
+ *
+ * \param name The name of the symbol
+ * \param ptr The symbol address.
+ * \return 0 when no error is thrown, -1 when failure happens
+ */
+TVM_DLL int TVMBackendRegisterEnvCAPI(const char* name, void* ptr);

Review comment:
       The particular API can be useful for future registrations. One potential candidate is PyInitialize, for initializing python callback support in multi-threaded setting
   
   




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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513


   @areusch There are usecases in AutoTIR where we want to call back to python in a multi-threaded C++ environment (with GIL). In this case, C++ threads need to be properly initialized by calling [several python's stable APIs](https://docs.python.org/3/c-api/init.html#non-python-created-threads). However, in fact we don't want to include `Python.h` because strictly speaking TVM is not designed to depend on python.
   
   Actually It's not a strict dependency, and we can completely avoid doing this in AutoTIR, but I'm just so curious, so TQ points me to this PR. After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,89 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  /*!
+   * \brief Clear the error indicator.
+   */
+  typedef void (*F_PyErr_Clear)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+  /*!
+   * \brief PyErrClear function
+   */
+  F_PyErr_Clear pyerr_clear = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {
+      Update(symbol_name, &pyerr_check_signals, fptr);
+    } else if (symbol_name == "PyErr_Clear") {
+      Update(symbol_name, &pyerr_clear, fptr);
+    } else {
+      LOG(FATAL) << "Unknown env API " << symbol_name;
+    }
+  }
+
+  // implementation of tvm::runtime::EnvCheckSignals
+  void CheckSignals() {
+    // check python signal to see if there are exception raised
+    if (pyerr_check_signals != nullptr && (*pyerr_check_signals)() != 0) {
+      ICHECK_NOTNULL(pyerr_clear);
+      // clear the error so we can throw a new error
+      (*pyerr_clear)();
+      // Raise the error
+      LOG(FATAL) << "KeyboardInterrupt: Signal received";

Review comment:
       Python should raise KeyboardInterrupt because the deferred signal handler does that, not because the external C function does




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

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



[GitHub] [tvm] tqchen commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @areusch I pushed an alternative implementation that can preserve the python exception types.  I started  with this one but then decided to go with the approach because most exceptions are indeed KeyboardInterrupt. After the discussion I agree it can be more important to preserve the exception type.
   
   This implementation does that. The approach is similar to pybind's approach of handling the exception. When error happens a special exception is thrown, and the error is not cleared but propagated into the FFI layer through a specific return code.
   
   The only drawback here is that the stacktrace will only start from the FFI(the C++ traces are not tracked), but it is fine as the signal handling itself is also already async.
   
   


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

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



[GitHub] [tvm] junrushao1994 commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   BTW I encountered something interesting where working on https://github.com/jwfromm/numpy_dlpack/: we need to replace CDLL with PyDLL to make Python-C extension calls working. I don’t find any good explanation but it seems to work this way. Ref: https://stackoverflow.com/questions/56470539/call-c-code-from-python-via-ctypes-use-python-object-list


-- 
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] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,89 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  /*!
+   * \brief Clear the error indicator.
+   */
+  typedef void (*F_PyErr_Clear)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+  /*!
+   * \brief PyErrClear function
+   */
+  F_PyErr_Clear pyerr_clear = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {
+      Update(symbol_name, &pyerr_check_signals, fptr);
+    } else if (symbol_name == "PyErr_Clear") {
+      Update(symbol_name, &pyerr_clear, fptr);
+    } else {
+      LOG(FATAL) << "Unknown env API " << symbol_name;
+    }
+  }
+
+  // implementation of tvm::runtime::EnvCheckSignals
+  void CheckSignals() {
+    // check python signal to see if there are exception raised
+    if (pyerr_check_signals != nullptr && (*pyerr_check_signals)() != 0) {
+      ICHECK_NOTNULL(pyerr_clear);
+      // clear the error so we can throw a new error
+      (*pyerr_clear)();
+      // Raise the error
+      LOG(FATAL) << "KeyboardInterrupt: Signal received";

Review comment:
       let's raise something besides KeyboardInterrupt. How about TVMSignalReceived or something

##########
File path: include/tvm/runtime/c_backend_api.h
##########
@@ -99,6 +100,19 @@ TVM_DLL void* TVMBackendAllocWorkspace(int device_type, int device_id, uint64_t
  */
 TVM_DLL int TVMBackendFreeWorkspace(int device_type, int device_id, void* ptr);
 
+/*!
+ * \brief Backend function to register execution environment(e.g. python)
+ *        specific C APIs.
+ *
+ * \note  We only register the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). In most cases we should use the PackedFunc FFI.
+ *
+ * \param name The name of the symbol
+ * \param ptr The symbol address.
+ * \return 0 when no error is thrown, -1 when failure happens
+ */
+TVM_DLL int TVMBackendRegisterEnvCAPI(const char* name, void* ptr);

Review comment:
       what's the motivation for doing this with a string name rather than just a dedicated API to handling signals? from a documentation perspective, what values could `name` take on, where are those documented, and when should you call this function? what happens if you don't register these?
   
   all of these questions make me think that maybe since this is quite an OS-specific thing, there's no need to consolidate similar things behind a single API call until we have multiple. On the other hand, I understand how this does avoid backwards-compatibility issues, and deprecating `.so` libraries does seem hard.

##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns

Review comment:
       "but will delay invoking the Python-side signal handler until the function returns to the Python side. This means that the effect of e.g. pressing Ctrl+C or sending signals the process will be delayed until function return. When a C function is blocked on a syscall such as accept(), it may become impossible to interrupt TVM outside of using SIGKILL. So this function is not needed in most API functions, which can finish quickly in a reasonable, deterministic amount of time"

##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,89 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  /*!
+   * \brief Clear the error indicator.
+   */
+  typedef void (*F_PyErr_Clear)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+  /*!
+   * \brief PyErrClear function
+   */
+  F_PyErr_Clear pyerr_clear = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {
+      Update(symbol_name, &pyerr_check_signals, fptr);
+    } else if (symbol_name == "PyErr_Clear") {
+      Update(symbol_name, &pyerr_clear, fptr);
+    } else {
+      LOG(FATAL) << "Unknown env API " << symbol_name;
+    }
+  }
+
+  // implementation of tvm::runtime::EnvCheckSignals
+  void CheckSignals() {
+    // check python signal to see if there are exception raised
+    if (pyerr_check_signals != nullptr && (*pyerr_check_signals)() != 0) {
+      ICHECK_NOTNULL(pyerr_clear);

Review comment:
       why not place PyErr_ stuff in the FFI layer and name the function FrontendHasPendingSignal

##########
File path: src/support/socket.h
##########
@@ -361,6 +363,41 @@ class Socket {
 #endif
   }
 
+  /*!
+   * \brief Call a function and retry if an EINTR error is encountered.
+   *
+   *  Socket operations can return EINTR when the interrupt handler
+   *  is registered by the execution environment(e.g. python).
+   *  We should retry if there is no KeyboardInterrupt recorded in
+   *  the environment.
+   *
+   * \note This function is needed to avoid rare interrupt event
+   *       in long running server code.
+   *
+   * \param func The function to retry.
+   * \return The return code returned by function f or error_value on retry failure.
+   */
+  template <typename FuncType>
+  ssize_t RetryCallOnEINTR(FuncType func) {
+    ssize_t ret = func();
+    // common path
+    if (ret != -1) return ret;
+    // less common path
+    do {
+      if (GetLastError() == EINTR) {
+        // Call into env check signals to see if there are
+        // environment specific(e.g. python) signal exceptions.
+        // This function will throw an exception if there is an KeyboardInterrupt.

Review comment:
       if the process received a signal that requires TVM to return immediately (e.g. SIGINT).

##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.

Review comment:
       This function is a no-op when TVMBackendRegisterCAPI("FrontendHasPendingSIgnal", ...) has not been called. (the parameter may need to be updated)

##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.
+ *
+ * \throws This function throws approperiate exception if an error happens,

Review comment:
       \throws This function throws an exception when the frontend FrontendHasPendingSignal indicates that a signal handler is pending. When this occurs, the intent of this function is to interrupt execution in TVM and return quickly to the calling frontend code.

##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function

Review comment:
       i think you could further refine this to:
   - on Linux, when siginterrupt() is set, invoke this function whenever a syscall returns EINTR. when it is not set, invoke it between long-running syscalls when you will not immediately return to the frontend.
   - on Windows, the same rules apply, but due to differences in signal processing, these are likely to only make a difference when used with Ctrl+C and socket calls
   
   the last bit is what I gather from reading docs. would be great if @rkimball could confirm

##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;

Review comment:
       check_signal_every_k_iter




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

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



[GitHub] [tvm] junrushao1994 commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @areusch There are usecases in AutoTIR where we want to call back to python's stable apis in a multi-threaded C++ environment. It's not a strict dependency, and actually we can completely avoid doing this, but I'm just so curious, so TQ points me to this PR.
   
   After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] junrushao1994 commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @areusch `PyMem_RawMalloc` is a minimalist example. Actually we might want to call `PyGILState_Ensure` and `PyGILState_Release` as indicated in python documentation here: https://docs.python.org/3/c-api/init.html#non-python-created-threads, so that we can call back to python without deadlocks in C++-created threads.
   
   We don't care about performance for now (if we do, we should completely get rid of python), and this is not a dependency to our project (we have already got rid of this behavior), just curious if such functionality could be done in TVM


-- 
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] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.
+ *
+ * \throws This function throws approperiate exception if an error happens,

Review comment:
       The main complexity of pushing the handling to FFI layer is that we will need to introduce specific compiled functions in the FFI layer(via cython and others)  complicates the implementation(we cannot use python to implement it because the handling have to go through the C API, which is doable but introduces complication to implement such function in cython without any support from the python land). 
   
   With additional effort we might be able to recover the exact exception raised. Given KeyboardInterrupt would be the most common one. We could change that to a generic exception indicating that an error happens during signal handling.
   
   I have also tried to propagate the exact error to the python side would be harder because it involves complications in the python calling boundary(we cannot do so in the python land and it have to be done in a C++ land). Directly normal return could raise an SystemError that also shows the original error message.
   
   Finally, back to the generic function. There are two ways to think about the handling. Given the only usecase so far is python I trid to avoid generalizing and centralizes the implementation in the runtime(to reduce indirection). We can refactor once we see more needs and given there is no API change such refactor should be easy
   




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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513


   @areusch There are usecases in AutoTIR where we want to call back to python's stable apis in a multi-threaded C++ environment. We don't want to include `Python.h` because TVM is not designed to depend on python.
   
   Actually It's not a strict dependency, and we can completely avoid doing this in AutoTIR, but I'm just so curious, so TQ points me to this PR.
   
   After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] tqchen commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   Thanks @areusch please take another look


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

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



[GitHub] [tvm] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/c_runtime_api.h
##########
@@ -32,6 +32,11 @@
  *  The common flow is:
  *   - Use TVMFuncListGlobalNames to get global function name
  *   - Use TVMFuncCall to call these functions.
+ *
+ *  Possible return values of the API functions:

Review comment:
       it would be nice to instead refer to an enum, similar to `tvm/runtime/crt/error_codes.h`

##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,76 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {

Review comment:
       constant?

##########
File path: include/tvm/runtime/logging.h
##########
@@ -198,6 +198,19 @@ class Error : public ::dmlc::Error {  // for backwards compatibility
   explicit Error(const std::string& s) : ::dmlc::Error(s) {}
 };
 
+/*!
+ * \brief Error message already set in frontend env.
+ *  This error can be thrown by EnvCheckSignals
+ */
+class EnvErrorAlreadySet : public ::dmlc::Error {

Review comment:
       as an error, seems like we could catch something named e.g. FrontendError. it should be possible to refer to a doc describing this by a canonical name to learn things like:
   - why is this error thrown?
   - who does it?
   - what is the generally-suggested cleanup/try-catch behavior when this happens?

##########
File path: include/tvm/runtime/c_runtime_api.h
##########
@@ -32,6 +32,11 @@
  *  The common flow is:
  *   - Use TVMFuncListGlobalNames to get global function name
  *   - Use TVMFuncCall to call these functions.
+ *
+ *  Possible return values of the API functions:
+ *  * 0: success
+ *  * -1: the error can be retrieved through TVMGetLastError.
+ *  * -2: a frontend error ocurred and recorded in the frontend.

Review comment:
       nit: sp: occurred

##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.

Review comment:
       can you just update the docstring to refer to specific names rather than the general concept of "signal checking function"?




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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-939576884






-- 
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] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513


   @areusch There are usecases in AutoTIR where we want to call back to python in a multi-threaded C++ environment. However, C++ threads need to be properly initialized by calling [several python's stable APIs](https://docs.python.org/3/c-api/init.html#non-python-created-threads), but in fact we don't want to include `Python.h` because strictly speaking TVM is not designed to depend on python.
   
   Actually It's not a strict dependency, and we can completely avoid doing this in AutoTIR, but I'm just so curious, so TQ points me to this PR.
   
   After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


-- 
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] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.
+ *
+ * \throws This function throws approperiate exception if an error happens,

Review comment:
       ah you are correct, my mistake. however, I don't agree with the implementation here. `PyErr_CheckSignals` states:
   
   >It checks whether a signal has been sent to the processes and if so, invokes the corresponding signal handler. If the signal module is supported, this can invoke a signal handler written in Python. In all cases, the default effect for SIGINT is to raise the KeyboardInterrupt exception. 
   
   If, in Python, I do: 
   ```
   old_sighandler = None
   def cleanup_temp_files(*args):
     shutil.rmtree("/tmp/foo")
     old_sighandler(*args)
   
   old_sighandler = signal.signal(signal.SIGINT, cleanup_temp_files)
   ```
   
   which is perfectly legal, but `shutil.rmtree` throws an exception, you will mask that exception here with KeyboardInterrupt. 
   
   Instead, it seems we need to throw a special C++-side exception which is caught in TVMFuncCall and then return normally to Python as if no error occurred (perhaps first asserting that `PyErr_Occurred() != nullptr`). this allows whatever exception was raised by the signal handler to propagate normally with correct traceback information. if for some reason `PyErr_Occurred() == nullptr` at this time, we could continue to raise a TVM-side error indicating that a signal handler interrupted the call. Perhaps the return value of TVMFuncCall should be modified to account for this scenario.
   
   In addition, I think it makes sense to push the handling of this into the FFI. Different frontends may handle this differently--for instance, other frontends may not be capable of executing the signal handler from a callback function such as `PyErr_CheckSignals`, so it seems like it makes sense to me merely to treat such a function as "should I continue?" and return an indicative error code when that function says no.




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,89 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  /*!
+   * \brief Clear the error indicator.
+   */
+  typedef void (*F_PyErr_Clear)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+  /*!
+   * \brief PyErrClear function
+   */
+  F_PyErr_Clear pyerr_clear = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {
+      Update(symbol_name, &pyerr_check_signals, fptr);
+    } else if (symbol_name == "PyErr_Clear") {
+      Update(symbol_name, &pyerr_clear, fptr);
+    } else {
+      LOG(FATAL) << "Unknown env API " << symbol_name;
+    }
+  }
+
+  // implementation of tvm::runtime::EnvCheckSignals
+  void CheckSignals() {
+    // check python signal to see if there are exception raised
+    if (pyerr_check_signals != nullptr && (*pyerr_check_signals)() != 0) {
+      ICHECK_NOTNULL(pyerr_clear);

Review comment:
       I have thought about these. Placing in FFI layer also have overheads(of introducing such function to the API). Given the two functions are clearly defined, I eventually decided to place them here as in their original form(to remove one level of indirection)




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

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



[GitHub] [tvm] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/c_runtime_api.h
##########
@@ -32,6 +32,11 @@
  *  The common flow is:
  *   - Use TVMFuncListGlobalNames to get global function name
  *   - Use TVMFuncCall to call these functions.
+ *
+ *  Possible return values of the API functions:

Review comment:
       it's more for organization to document the 0/-1/-2 convention. perhaps you're right...it would be nice to align on an error code type, but it could be a different PR




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: src/runtime/registry.cc
##########
@@ -102,6 +103,76 @@ std::vector<std::string> Registry::ListNames() {
   return keys;
 }
 
+/*!
+ * \brief Execution environment specific API registry.
+ *
+ *  This registry stores C API function pointers about
+ *  execution environment(e.g. python) specific API function that
+ *  we need for specific low-level handling(e.g. signal checking).
+ *
+ *  We only stores the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). Always consider use the PackedFunc FFI when possible
+ *  in other cases.
+ */
+class EnvCAPIRegistry {
+ public:
+  /*!
+   * \brief Callback to check if signals have been sent to the process and
+   *        if so invoke the registered signal handler in the frontend environment.
+   *
+   *  When runnning TVM in another langugage(python), the signal handler
+   *  may not be immediately executed, but instead the signal is marked
+   *  in the interpreter state(to ensure non-blocking of the signal handler).
+   *
+   * \return 0 if no error happens, -1 if error happens.
+   */
+  typedef int (*F_PyErr_CheckSignals)();
+
+  // NOTE: the following function are only registered
+  // in a python environment.
+  /*!
+   * \brief PyErr_CheckSignal function
+   */
+  F_PyErr_CheckSignals pyerr_check_signals = nullptr;
+
+  static EnvCAPIRegistry* Global() {
+    static EnvCAPIRegistry* inst = new EnvCAPIRegistry();
+    return inst;
+  }
+
+  // register environment(e.g. python) specific api functions
+  void Register(const std::string& symbol_name, void* fptr) {
+    if (symbol_name == "PyErr_CheckSignals") {

Review comment:
       given the string is only used in a single place, I believe this way is more readable.




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

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



[GitHub] [tvm] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.
+ *
+ * \throws This function throws approperiate exception if an error happens,

Review comment:
       > The main complexity of pushing the handling to FFI layer is that we will need to introduce specific compiled functions in the FFI layer
   
   In this case, the functions are designed to be used with the FFI layer. So I'm not sure I see a problem with moving the use of the functions closer to the place it's intended to live.
   
   In this case, the complexity is actually the same, because you remove the need to expose PyErr_Clear() to TVM and instead replace its use with asserting `PyError_Occurred() != nullptr`. The complexity in the TVM runtime is then reduced to that around the single frontend-agnostic function which asks "should I continue?"
   
   Could you clarify about "I have also tried to propagate the exact error to the python side?" Theoretically, you should not need to propagate any error here--Python should take care of that.
   
   I'm okay with how we treat the generic function now, though I do think that if there is just one, it may be a bit overcomplicated to introduce a registry. A single PackedFunc should suffice here.




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/c_backend_api.h
##########
@@ -99,6 +100,19 @@ TVM_DLL void* TVMBackendAllocWorkspace(int device_type, int device_id, uint64_t
  */
 TVM_DLL int TVMBackendFreeWorkspace(int device_type, int device_id, void* ptr);
 
+/*!
+ * \brief Backend function to register execution environment(e.g. python)
+ *        specific C APIs.
+ *
+ * \note  We only register the C API function when absolutely necessary (e.g. when signal handler
+ *  cannot trap back into python). In most cases we should use the PackedFunc FFI.
+ *
+ * \param name The name of the symbol
+ * \param ptr The symbol address.
+ * \return 0 when no error is thrown, -1 when failure happens
+ */
+TVM_DLL int TVMBackendRegisterEnvCAPI(const char* name, void* ptr);

Review comment:
       Right the main rationale is to make the C API minimal. Note that two functions are already needed(PyErr_GetSignals and PyErr_Clear)




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

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



[GitHub] [tvm] areusch edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
areusch edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-943495292


   @junrushao1994 ok, if you want to do this i'd strongly encourage you to consider whether it's worth the complexity of multi-threading in c++ (it certainly could be, but the GIL is a pretty easy way to nix any threading benefits :) and Python can quickly destroy any cache locality you may have leveraged in multi-threaded apps). 
   
   i am guessing you want to call `PyMem_RawMalloc`? I do see in the [docs](https://docs.python.org/3/c-api/memory.html#raw-memory-interface): 
   > The following function sets are wrappers to the system allocator. These functions are thread-safe, the GIL does not need to be held.
   
   does your question still apply to this specific use case? in general, this callback function, although named due to Python, isn't so specific to Python and is more specific to syscall restart behavior and signal processing.


-- 
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] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.
+ *
+ * \throws This function throws approperiate exception if an error happens,

Review comment:
       This is not accurate. The error only happens when signal handler throws an error, will update




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/c_runtime_api.h
##########
@@ -32,6 +32,11 @@
  *  The common flow is:
  *   - Use TVMFuncListGlobalNames to get global function name
  *   - Use TVMFuncCall to call these functions.
+ *
+ *  Possible return values of the API functions:

Review comment:
       Given the number of categories are small atm, I feel it may not be necessary to introduce an error code enum




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

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



[GitHub] [tvm] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/registry.h
##########
@@ -89,8 +89,7 @@ namespace runtime {
  *
  * \endcode
  *
- * \note This function is a nop when no signal checking function is registered.
- *       In particular PyErr_CheckSignals and PyErr_Clear when running in python.
+ * \note This function is a nop when no PyErr_CheckSignal is registered.

Review comment:
       (if you make another rev) nit: PyErr_CheckSignals, to improve grepping




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

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



[GitHub] [tvm] areusch commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @junrushao1994 just curious what the motivation is behind your question?


-- 
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] areusch commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   ok, if you want to do this i'd strongly encourage you to consider whether it's worth the complexity of multi-threading in c++ (it certainly could be, but the GIL is a pretty easy way to nix any threading benefits :) and Python can quickly destroy any cache locality you may have leveraged in multi-threaded apps). 
   
   i am guessing you want to call `PyMem_RawMalloc`? I do see in the [docs](https://docs.python.org/3/c-api/memory.html#raw-memory-interface): 
   > The following function sets are wrappers to the system allocator. These functions are thread-safe, the GIL does not need to be held.
   
   does your question still apply to this specific use case? in general, this callback function, although named due to Python, isn't so specific to Python and is more specific to syscall restart behavior and signal processing.


-- 
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] areusch commented on a change in pull request #7919: [RUNTIME] Improve signal handling in python env.

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



##########
File path: include/tvm/runtime/registry.h
##########
@@ -52,6 +52,45 @@
 namespace tvm {
 namespace runtime {
 
+/*!
+ * \brief Check if signals have been sent to the process and if so
+ *  invoke the registered signal handler in the frontend environment.
+ *
+ *  When runnning TVM in another langugage(python), the signal handler
+ *  may not be immediately executed, but instead the signal is marked
+ *  in the interpreter state(to ensure non-blocking of the signal handler).
+ *
+ *  This function can be explicitly invoked to check the cached signal
+ *  and run the related processing if a signal is marked.
+ *
+ *  Invoke this function periodically in a long running C++ function
+ *  to check if KeyboardInterrupt happens in a python execution environment.
+ *
+ *  Not inserting this function will not cause any correctness
+ *  issue, but will delay the KeyboardInterrupt until the function returns
+ *  to the python side. So this function is not needed in most API
+ *  functions can finish quickly in a reasonable amount of time.
+ *
+ * \code
+ *
+ * int check_signal_every_kiter = 10;
+ *
+ * for (int iter = 0; iter < very_large_number; ++iter) {
+ *   if (iter % check_signal_every_kiter == 0) {
+ *     tvm::runtime::EnvCheckSignals();
+ *   }
+ *   // do work here
+ * }
+ *
+ * \endcode
+ *
+ * \note This function is a nop when no signal checking function is registered.
+ *
+ * \throws This function throws approperiate exception if an error happens,

Review comment:
       > The main complexity of pushing the handling to FFI layer is that we will need to introduce specific compiled functions in the FFI layer
   
   In this case, the functions are designed to be used with the FFI layer. So I'm not sure I see a problem with moving the use of the functions closer to the place where they are intended to be used.
   
   In this case, the complexity is actually the same, because you remove the need to expose PyErr_Clear() to TVM and instead replace its use with asserting `PyError_Occurred() != nullptr`. The complexity in the TVM runtime is then reduced to that around the single frontend-agnostic function which asks "should I continue?"
   
   Could you clarify about "I have also tried to propagate the exact error to the python side?" Theoretically, you should not need to propagate any error here--Python should take care of that.
   
   I'm okay with how we treat the generic function now, though I do think that if there is just one, it may be a bit overcomplicated to introduce a registry. A single PackedFunc should suffice here.




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

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



[GitHub] [tvm] tqchen merged pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   


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

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



[GitHub] [tvm] tqchen commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   Thanks @areusch I tried to incorporate most of your comments, please take another look


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

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



[GitHub] [tvm] junrushao1994 commented on pull request #7919: [RUNTIME] Improve signal handling in python env.

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


   @tqchen while this approach is more specific to cython, I would love to discuss the possibility about a ctype-based generic approach to register arbitrary python api with `TVMBackendRegisterEnvCAPI`:
   
   It seems that `ctypes.pythonapi.*` provides all the python C apis. For example:
   - `ctypes.pythonapi.Py_IncRef` corresponds to `Py_IncRef` in Python.h
   - `ctypes.pythonapi.Py_DecRef` corresponds to `Py_DecRef` in Python.h
   - `ctypes.pythonapi.PyMem_RawMalloc` corresponds to `PyMem_RawMalloc` in Python.h
   - `ctypes.pythonapi.PyMem_RawFree` corresponds to `PyMem_RawFree` in Python.h
   
   Therefore, instead of registering cython method, what about we register apis under `ctypes.pythonapi` after CDLL is loaded with ctypes? This way, it could work for both ctypes and cython, and probably generalizable to future work to call more python apis without having to depend on Python.h.
   
   CC: @zxybazh 


-- 
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] junrushao1994 edited a comment on pull request #7919: [RUNTIME] Improve signal handling in python env.

Posted by GitBox <gi...@apache.org>.
junrushao1994 edited a comment on pull request #7919:
URL: https://github.com/apache/tvm/pull/7919#issuecomment-941683513


   @areusch There are usecases in AutoTIR where we want to call back to python in a multi-threaded C++ environment. However, C++ threads need to be properly initialized by calling [several python's stable APIs](https://docs.python.org/3/c-api/init.html#non-python-created-threads), but in fact we don't want to include `Python.h` because strictly speaking TVM is not designed to depend on python.
   
   Actually It's not a strict dependency, and we can completely avoid doing this in AutoTIR, but I'm just so curious, so TQ points me to this PR. After reading this, I sort of realized that this PR only works for cython cases, so was curious how to do it correctly using ctypes. 


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