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

[GitHub] [tvm] zxybazh opened a new pull request #10368: [WIP][MetaSchedule] Fix Cyclic Dependency in PyClass Family

zxybazh opened a new pull request #10368:
URL: https://github.com/apache/tvm/pull/10368


   Following the design of `module_pass`, we developed a mechanism, a decorator named `derived_obj`, to systematically allow derivation from TVM objects in pure Python and being passed into any language, without cyclic dependency. This PR introduces the new mechanism to all PyClasses in meta schedule.
   
   Class Migration Progress:
   
   - [x] TaskScheduler (allows hybrid mode)
   - [x] SpaceGenerator
   - [x] SearchStrategy
   - [x] Builder
   - [x] Runner
   - [x] RunnerFuture
   - [ ] Database
   - [ ] CostModel
   - [ ] Mutator
   - [ ] Postproc
   - [ ] ScheduleRule
   - [ ] FeatureExtractor
   - [ ] MeasureCallback


-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       This one should work:
   
   ```python
   Object(self.handle)
   ```




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       object(self.handle)




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/task_scheduler/task_scheduler.py
##########
@@ -187,10 +160,114 @@ def f_join_running_task(task_id: int) -> None:
             database,
             cost_model,
             measure_callbacks,
-            f_tune,
-            f_initialize_task,
-            f_set_task_stopped,
-            f_is_task_running,
-            f_join_running_task,
-            f_next_task_id,
+            *methods,
         )
+
+
+class PyTaskScheduler:
+    """
+    An abstract task scheduler with customized methods on the python-side.
+    This is the user facing class for function overloading inheritance.
+
+    Note: @derived_object is required for proper usage of any inherited class.
+    """
+
+    _tvm_metadata = {
+        "cls": _PyTaskScheduler,
+        "members": [
+            "tasks",
+            "builder",
+            "runner",
+            "database",
+            "cost_model",
+            "measure_callbacks",
+        ],
+        "methods": [
+            "tune",
+            "initialize_task",
+            "set_task_stopped",
+            "is_task_running",
+            "join_running_task",
+            "next_task_id",
+        ],
+    }
+
+    def __init__(
+        self,
+        tasks: List[TuneContext],
+        builder: Builder,
+        runner: Runner,
+        database: Database,
+        cost_model: Optional[CostModel] = None,
+        measure_callbacks: Optional[List[MeasureCallback]] = None,
+    ):
+        self.tasks = tasks
+        self.builder = builder
+        self.runner = runner
+        self.database = database
+        self.cost_model = cost_model
+        self.measure_callbacks = measure_callbacks
+
+    def tune(self) -> None:
+        """Auto-tuning."""
+        # Using self._outer to replace the self pointer
+        _ffi_api.TaskSchedulerTune(self._outer())  # type: ignore # pylint: disable=no-member

Review comment:
       Thanks for the advice. In this special case since we have to call back to c++ function with a reference to the object we might not be able to do it the same way. I prefer to keep the current way to preserve the outside shell class weakref pointer. Also I've added a unittest at `tests/python/unittest/test_meta_schedule_task_scheduler.py` to make sure the object can be destructed without cyclic dependency.




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       Is there any particular reason to hold weakref to access the outer class?




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )

Review comment:
       Let's deliberate the constructor design. Maybe it could look clearer if we add a star here:
   
   
   ```suggestion
               super().__init__(
                   # the constructor's parameters, builder, runner, etc.
                   *[getattr(self._inst, name) for name in members],
                   # the function methods, init_with_tune_context, build, run, etc.
                   *[_extract(self._inst, name) for name in methods],
               )
   ```
   
   and in the corresponding constructor of the base class we instead use:
   
   ```python
   
   @register_object("meta_schedule.PyRunner")
   class _PyRunner(meta_schedule.Runner):
       def __init__(self, field_1, field_2, method_1, method_2):
           self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, field_1, field_2, method_1, method_2)
   ```
   




-- 
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 #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/task_scheduler/task_scheduler.py
##########
@@ -187,10 +160,114 @@ def f_join_running_task(task_id: int) -> None:
             database,
             cost_model,
             measure_callbacks,
-            f_tune,
-            f_initialize_task,
-            f_set_task_stopped,
-            f_is_task_running,
-            f_join_running_task,
-            f_next_task_id,
+            *methods,
         )
+
+
+class PyTaskScheduler:
+    """
+    An abstract task scheduler with customized methods on the python-side.
+    This is the user facing class for function overloading inheritance.
+
+    Note: @derived_object is required for proper usage of any inherited class.
+    """
+
+    _tvm_metadata = {
+        "cls": _PyTaskScheduler,
+        "members": [
+            "tasks",
+            "builder",
+            "runner",
+            "database",
+            "cost_model",
+            "measure_callbacks",
+        ],
+        "methods": [
+            "tune",
+            "initialize_task",
+            "set_task_stopped",
+            "is_task_running",
+            "join_running_task",
+            "next_task_id",
+        ],
+    }
+
+    def __init__(
+        self,
+        tasks: List[TuneContext],
+        builder: Builder,
+        runner: Runner,
+        database: Database,
+        cost_model: Optional[CostModel] = None,
+        measure_callbacks: Optional[List[MeasureCallback]] = None,
+    ):
+        self.tasks = tasks
+        self.builder = builder
+        self.runner = runner
+        self.database = database
+        self.cost_model = cost_model
+        self.measure_callbacks = measure_callbacks
+
+    def tune(self) -> None:
+        """Auto-tuning."""
+        # Using self._outer to replace the self pointer
+        _ffi_api.TaskSchedulerTune(self._outer())  # type: ignore # pylint: disable=no-member

Review comment:
       Assigning handle to inst is kinda of similar to weakref already(so that is not too much of a difference as long as we confirm weakref works).  The main reason was in your inst class implementation there is a need to pass in the object itself.
   
   One possible alternative is to let tune to take a second parameter that records as outer. Then have the wrapper class pass the function in. 
   
   ```python
   def tune(self, outer):
       pass
   ```
   The weakref might be an OK approach here if we property document it.
   
   Alternatively, do not implement methods like tune in PyTaskScheduler, and expose a TaskScheduler object(that corresponds to the C++ object) which exposes the FFI function.  Just like https://github.com/apache/tvm/blob/main/python/tvm/ir/transform.py#L160
   
   Then the derived class only need to specify some of the related methods like next_task_id, get wrapped as a TaskScheduler which comes with the tune method
   
   
   
   




-- 
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 #10368: [WIP][MetaSchedule] Fix Cyclic Dependency in PyClass Family

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


   CC @YuchenJin @sunggg would love to have you guys review this PR :-) Thanks a lot!


-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/task_scheduler/task_scheduler.py
##########
@@ -187,10 +160,114 @@ def f_join_running_task(task_id: int) -> None:
             database,
             cost_model,
             measure_callbacks,
-            f_tune,
-            f_initialize_task,
-            f_set_task_stopped,
-            f_is_task_running,
-            f_join_running_task,
-            f_next_task_id,
+            *methods,
         )
+
+
+class PyTaskScheduler:
+    """
+    An abstract task scheduler with customized methods on the python-side.
+    This is the user facing class for function overloading inheritance.
+
+    Note: @derived_object is required for proper usage of any inherited class.
+    """
+
+    _tvm_metadata = {
+        "cls": _PyTaskScheduler,
+        "members": [
+            "tasks",
+            "builder",
+            "runner",
+            "database",
+            "cost_model",
+            "measure_callbacks",
+        ],
+        "methods": [
+            "tune",
+            "initialize_task",
+            "set_task_stopped",
+            "is_task_running",
+            "join_running_task",
+            "next_task_id",
+        ],
+    }
+
+    def __init__(
+        self,
+        tasks: List[TuneContext],
+        builder: Builder,
+        runner: Runner,
+        database: Database,
+        cost_model: Optional[CostModel] = None,
+        measure_callbacks: Optional[List[MeasureCallback]] = None,
+    ):
+        self.tasks = tasks
+        self.builder = builder
+        self.runner = runner
+        self.database = database
+        self.cost_model = cost_model
+        self.measure_callbacks = measure_callbacks
+
+    def tune(self) -> None:
+        """Auto-tuning."""
+        # Using self._outer to replace the self pointer
+        _ffi_api.TaskSchedulerTune(self._outer())  # type: ignore # pylint: disable=no-member

Review comment:
       Note that we need the reference to it's shell class, i.e., `TVMDrivedObj` and we also need to avoid cyclic dependency. Other than using the weakref, we can only use the given handle pointer. Is there any way that we can create the object directly from given pointer in python?




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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None

Review comment:
       Why `__str__` is the special case btw?




-- 
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] YuchenJin commented on a change in pull request #10368: [WIP][MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: Any) -> type:

Review comment:
       cls: type?




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       ^Tons of `Segmentation fault`.




-- 
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 #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/task_scheduler/task_scheduler.py
##########
@@ -187,10 +160,114 @@ def f_join_running_task(task_id: int) -> None:
             database,
             cost_model,
             measure_callbacks,
-            f_tune,
-            f_initialize_task,
-            f_set_task_stopped,
-            f_is_task_running,
-            f_join_running_task,
-            f_next_task_id,
+            *methods,
         )
+
+
+class PyTaskScheduler:
+    """
+    An abstract task scheduler with customized methods on the python-side.
+    This is the user facing class for function overloading inheritance.
+
+    Note: @derived_object is required for proper usage of any inherited class.
+    """
+
+    _tvm_metadata = {
+        "cls": _PyTaskScheduler,
+        "members": [
+            "tasks",
+            "builder",
+            "runner",
+            "database",
+            "cost_model",
+            "measure_callbacks",
+        ],
+        "methods": [
+            "tune",
+            "initialize_task",
+            "set_task_stopped",
+            "is_task_running",
+            "join_running_task",
+            "next_task_id",
+        ],
+    }
+
+    def __init__(
+        self,
+        tasks: List[TuneContext],
+        builder: Builder,
+        runner: Runner,
+        database: Database,
+        cost_model: Optional[CostModel] = None,
+        measure_callbacks: Optional[List[MeasureCallback]] = None,
+    ):
+        self.tasks = tasks
+        self.builder = builder
+        self.runner = runner
+        self.database = database
+        self.cost_model = cost_model
+        self.measure_callbacks = measure_callbacks
+
+    def tune(self) -> None:
+        """Auto-tuning."""
+        # Using self._outer to replace the self pointer
+        _ffi_api.TaskSchedulerTune(self._outer())  # type: ignore # pylint: disable=no-member

Review comment:
       Assigning handle to inst is kinda of similar to weakref already.  The main reason was in your inst class implementation there is a need to pass in the object itself.
   
   One possible alternative is to let tune to take a second parameter that records as outer. Then have the wrapper class pass the function in. 
   
   ```python
   def tune(self, outer):
       pass
   ```
   The weakref might be an OK approach here if we property document it.
   
   Alternatively, do not implement methods like tune in PyTaskScheduler, and expose a TaskScheduler object(that corresponds to the C++ object) which exposes the FFI function.  Just like https://github.com/apache/tvm/blob/main/python/tvm/ir/transform.py#L160
   
   Then the derived class only need to specify some of the related methods like next_task_id, get wrapped as a TaskScheduler which comes with the tune method
   
   
   
   




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: Any) -> type:

Review comment:
       Fixed.
   




-- 
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] YuchenJin commented on a change in pull request #10368: [WIP][MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: Any) -> type:

Review comment:
       Just curious, is `derived_object` generic enough to work for all tvm Objects?




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       I did not find a very convenient way to creat object from given `c_void_p`, i.e., `self.handle`, so I kept the reference here for later usage. And suggestion?

##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       I did not find a very convenient way to creat object from given `c_void_p`, i.e., `self.handle`, so I kept the reference here for later usage. And suggestions?




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/task_scheduler/task_scheduler.py
##########
@@ -187,10 +160,114 @@ def f_join_running_task(task_id: int) -> None:
             database,
             cost_model,
             measure_callbacks,
-            f_tune,
-            f_initialize_task,
-            f_set_task_stopped,
-            f_is_task_running,
-            f_join_running_task,
-            f_next_task_id,
+            *methods,
         )
+
+
+class PyTaskScheduler:
+    """
+    An abstract task scheduler with customized methods on the python-side.
+    This is the user facing class for function overloading inheritance.
+
+    Note: @derived_object is required for proper usage of any inherited class.
+    """
+
+    _tvm_metadata = {
+        "cls": _PyTaskScheduler,
+        "members": [
+            "tasks",
+            "builder",
+            "runner",
+            "database",
+            "cost_model",
+            "measure_callbacks",
+        ],
+        "methods": [
+            "tune",
+            "initialize_task",
+            "set_task_stopped",
+            "is_task_running",
+            "join_running_task",
+            "next_task_id",
+        ],
+    }
+
+    def __init__(
+        self,
+        tasks: List[TuneContext],
+        builder: Builder,
+        runner: Runner,
+        database: Database,
+        cost_model: Optional[CostModel] = None,
+        measure_callbacks: Optional[List[MeasureCallback]] = None,
+    ):
+        self.tasks = tasks
+        self.builder = builder
+        self.runner = runner
+        self.database = database
+        self.cost_model = cost_model
+        self.measure_callbacks = measure_callbacks
+
+    def tune(self) -> None:
+        """Auto-tuning."""
+        # Using self._outer to replace the self pointer
+        _ffi_api.TaskSchedulerTune(self._outer())  # type: ignore # pylint: disable=no-member

Review comment:
       I don't think we should use the weakref dark magic




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       This one should work:
   
   ```python
   Object(self.handle)
   ```
   
   if it doesn't, try this one:
   
   ```python
   Object.__new__(self.handle)
   ```




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       _PyRunner(self.handle)




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: Any) -> type:

Review comment:
       Note that derived object works with non-tvm objects, e.g., `PyRunner`, while it generates a new class called `TVMDerivedObj` inheriting from the `_tvm_metadata["cls"]` field of the non-tvm object, e.g., `_PyRunner`. By design the latter base class for inheritance can be any tvm object.




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       I did not found a convenient way to creat object from given `c_void_p`, i.e., `self.handle`, so I kept the reference here for later usage. And suggestion?




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       could we try this?
   
   ```
   from tvm.runtime.object import ObjectBase
   
   obj = ObjectBase()
   obj.handle = self.handle
   ```

##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )
+
+            # for task scheduler hybrid funcs in c++ & python side
+            # using weakref to avoid cyclic dependency
+            self._inst._outer = weakref.ref(self)

Review comment:
       could we try this?
   
   ```python
   from tvm.runtime.object import ObjectBase
   
   obj = ObjectBase()
   obj.handle = self.handle
   ```




-- 
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 #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/task_scheduler/task_scheduler.py
##########
@@ -187,10 +160,114 @@ def f_join_running_task(task_id: int) -> None:
             database,
             cost_model,
             measure_callbacks,
-            f_tune,
-            f_initialize_task,
-            f_set_task_stopped,
-            f_is_task_running,
-            f_join_running_task,
-            f_next_task_id,
+            *methods,
         )
+
+
+class PyTaskScheduler:
+    """
+    An abstract task scheduler with customized methods on the python-side.
+    This is the user facing class for function overloading inheritance.
+
+    Note: @derived_object is required for proper usage of any inherited class.
+    """
+
+    _tvm_metadata = {
+        "cls": _PyTaskScheduler,
+        "members": [
+            "tasks",
+            "builder",
+            "runner",
+            "database",
+            "cost_model",
+            "measure_callbacks",
+        ],
+        "methods": [
+            "tune",
+            "initialize_task",
+            "set_task_stopped",
+            "is_task_running",
+            "join_running_task",
+            "next_task_id",
+        ],
+    }
+
+    def __init__(
+        self,
+        tasks: List[TuneContext],
+        builder: Builder,
+        runner: Runner,
+        database: Database,
+        cost_model: Optional[CostModel] = None,
+        measure_callbacks: Optional[List[MeasureCallback]] = None,
+    ):
+        self.tasks = tasks
+        self.builder = builder
+        self.runner = runner
+        self.database = database
+        self.cost_model = cost_model
+        self.measure_callbacks = measure_callbacks
+
+    def tune(self) -> None:
+        """Auto-tuning."""
+        # Using self._outer to replace the self pointer
+        _ffi_api.TaskSchedulerTune(self._outer())  # type: ignore # pylint: disable=no-member

Review comment:
       Assigning handle to inst is kinda of similar to weakref already(so that is not too much of a difference as long as we confirm weakref works).  The main reason was in your inst class implementation there is a need to pass in the object itself.
   
   One possible alternative is to let tune to take a second parameter that records as outer. Then have the wrapper class pass the function in. 
   
   ```python
   def tune(self, outer):
       pass
   ```
   The weakref might be an OK approach here if we property document it.
   
   
   Alternatively, do not implement methods like tune in PyTaskScheduler, and expose a TaskScheduler object(that corresponds to the C++ object) which exposes the FFI function.  Just like how ModulePass is exposed as an object, while the decorator returns an object that is represented as a ModulePass https://github.com/apache/tvm/blob/main/python/tvm/ir/transform.py#L160
   
   Then the derived class only need to specify some of the related methods like next_task_id, get wrapped as a TaskScheduler which comes with the tune method(Note that because TaskSchedule corresponds to the C++ class, passing self is the intended approach there and no outer is needed)
   
   




-- 
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 a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }

Review comment:
       "members" => "fields"
   
   ```suggestion
                   "cls": _PyRunner,
                   "fields": [],
                   "methods": ["run"],
               }
   ```




-- 
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] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None

Review comment:
       Because `__str__` does not need to be overrided, it's guaranteed to work.




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

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

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



[GitHub] [tvm] zxybazh commented on a change in pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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



##########
File path: python/tvm/meta_schedule/utils.py
##########
@@ -31,6 +31,107 @@
 from tvm.tir import FloatImm, IntImm
 
 
+def derived_object(cls: type) -> type:
+    """A decorator to register derived subclasses for TVM objects.
+
+    Parameters
+    ----------
+    cls : type
+        The derived class to be registered.
+
+    Returns
+    -------
+    cls : type
+        The decorated TVM object.
+
+    Example
+    -------
+    .. code-block:: python
+
+        @register_object("meta_schedule.PyRunner")
+        class _PyRunner(meta_schedule.Runner):
+            def __init__(self, methods: List[Callable]):
+                self.__init_handle_by_constructor__(_ffi_api.RunnerPyRunner, *methods)
+
+        class PyRunner():
+            _tvm_metadata = {
+                "cls": _PyRunner,
+                "methods": ["run"]
+            }
+            def run(self, runner_inputs):
+                raise NotImplementedError
+
+        @derived_object
+        class LocalRunner(PyRunner):
+            def run(self, runner_inputs):
+                ...
+    """
+
+    import functools  # pylint: disable=import-outside-toplevel
+    import weakref  # pylint: disable=import-outside-toplevel
+
+    def _extract(inst: type, name: str):
+        """Extract function from intrinsic class."""
+
+        def method(*args, **kwargs):
+            return getattr(inst, name)(*args, **kwargs)
+
+        if getattr(base, name) is getattr(cls, name) and name != "__str__":
+            # for task scheduler return None means calling default function
+            # otherwise it will trigger a TVMError of method not implemented
+            # on the c++ side when you call the method, __str__ not required
+            return None
+        return method
+
+    assert isinstance(cls.__base__, type)
+    assert hasattr(
+        cls, "_tvm_metadata"
+    ), "Please use the user-facing method overiding class, i.e., PyRunner."
+
+    base = cls.__base__
+    metadata = getattr(base, "_tvm_metadata")
+    members = metadata.get("members", [])
+    methods = metadata.get("methods", [])
+
+    class TVMDerivedObject(metadata["cls"]):  # type: ignore
+        """The derived object to avoid cyclic dependency."""
+
+        def __init__(self, *args, **kwargs):
+            """Constructor."""
+            self.handle = None
+            self._inst = cls(*args, **kwargs)
+
+            super().__init__(
+                # the constructor's parameters, builder, runner, etc.
+                *[getattr(self._inst, name) for name in members],
+                # the function methods, init_with_tune_context, build, run, etc.
+                [_extract(self._inst, name) for name in methods],
+            )

Review comment:
       Fixed.




-- 
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 merged pull request #10368: [MetaSchedule] Fix Cyclic Dependency in PyClass Family

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


   


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