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/06/14 07:14:31 UTC

[GitHub] [tvm] areusch commented on a change in pull request #8220: [DOCS] Add docs for Pass Instrument

areusch commented on a change in pull request #8220:
URL: https://github.com/apache/tvm/pull/8220#discussion_r650085612



##########
File path: docs/dev/pass_infra.rst
##########
@@ -389,6 +396,103 @@ To allow other C++ modules to apply this pass, we declare a free function in
 
     TVM_DLL Pass FoldConstant();
 
+.. _pass_instrument_section_tag:
+
+Pass Instrument
+~~~~~~~~~~~~~~~
+
+``PassInstrument`` provides callbacks run when entering/exiting ``PassContext`` and before/after executing passes.
+Multiple ``PassInstrument`` instances can be registed into a single ``PassContext``.
+Instrument instances are called sequentially in the order of ``instruments`` argument passed to ``PassContext``.
+
+.. code:: c++
+
+    namespace instrument {
+
+    class PassInstrumentNode : public Object {
+     public:
+      String name;
+      virtual void EnterPassContext() const = 0;
+      virtual void ExitPassContext() const = 0;
+      virtual bool ShouldRun(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      virtual void RunBeforePass(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      virtual void RunAfterPass(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      /* Other fields are omitted. */
+    };
+
+    class PassInstrument : public ObjectRef {
+     public:
+      TVM_DEFINE_OBJECT_REF_METHODS(PassInstrument, ObjectRef, PassInstrumentNode);
+    };
+
+    }  // namespace instrument
+
+Python interfaces are provided to implement ``PassInstrument`` quickly.
+
+Following four methods are invoked in the life-cycle of ``PassContext``.
+
+.. code:: c++
+
+    TVM_DLL void InstrumentEnterPassContext();
+    TVM_DLL void InstrumentExitPassContext();
+    TVM_DLL bool InstrumentBeforePass(const IRModule& mod, const PassInfo& info) const;
+    TVM_DLL void InstrumentAfterPass(const IRModule& mod, const PassInfo& info) const;
+
+``InstrumentEnterPassContext`` is called immediately when the scope
+of the ``PassContext`` instance is entered.
+
+``InstrumentExitPassContext`` is called when the scope of ``PassContextNode``
+is being leaved, or exceptions occur during the execution of passes.
+This method is also called when instruments is being overriden by ``override_instruments`` in ::py:class:`tvm.transform.PassContext`.
+
+``InstrumentBeforePass`` is called before pass-execution.
+``InstrumentAfterPass`` is called after pass-executioon if the pass should be run. The behavir is like:

Review comment:
       nit: execution, behavior

##########
File path: docs/dev/pass_infra.rst
##########
@@ -389,6 +396,103 @@ To allow other C++ modules to apply this pass, we declare a free function in
 
     TVM_DLL Pass FoldConstant();
 
+.. _pass_instrument_section_tag:
+
+Pass Instrument

Review comment:
       it seems like PassContext may be introduced after this section, no? Perhaps it makes sense to move it downward.

##########
File path: docs/dev/pass_infra.rst
##########
@@ -389,6 +396,103 @@ To allow other C++ modules to apply this pass, we declare a free function in
 
     TVM_DLL Pass FoldConstant();
 
+.. _pass_instrument_section_tag:
+
+Pass Instrument
+~~~~~~~~~~~~~~~
+
+``PassInstrument`` provides callbacks run when entering/exiting ``PassContext`` and before/after executing passes.
+Multiple ``PassInstrument`` instances can be registed into a single ``PassContext``.
+Instrument instances are called sequentially in the order of ``instruments`` argument passed to ``PassContext``.

Review comment:
       Don't abbreviate class names, it will get very confusing: s/Instrument/PassInstrument/

##########
File path: tutorials/dev/use_pass_instrument.py
##########
@@ -0,0 +1,229 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=line-too-long
+"""
+.. _tutorial-use-pass-instrument:
+
+How to Use TVM Pass Instrument
+==============================
+**Author**: `Chi-Wei Wang <https://github.com/chiwwang>`_
+
+As more and more passes are implemented, it becomes useful to instrument
+passes execution, analyze per-pass effects and observe various events.
+Pass infrastructure provides instrument mechanism. One can pass a list of
+instrument instances to :py:class:`tvm.transform.PassContext`.
+Also a decorator :py:func:`tvm.instrument.pass_instrument` is provided
+to easily implement instrument classes.
+
+This tutorial demostrates how developers can use ``PassContext`` to instrument
+passes. Please also refer to the :ref:`pass-infra`.
+"""
+import tvm
+import tvm.relay as relay
+from tvm.relay.testing import resnet
+from tvm.contrib.download import download_testdata
+from tvm.relay.build_module import bind_params_by_name
+from tvm.ir.instrument import (
+    PassTimingInstrument,
+    pass_instrument,
+)
+
+
+###############################################################################
+# Create An Example Relay Program
+# -------------------------------
+# We use pre-defined resnet-18 network in Relay.
+batch_size = 1
+num_of_image_class = 1000
+image_shape = (3, 224, 224)
+output_shape = (batch_size, num_of_image_class)
+relay_mod, relay_params = resnet.get_workload(num_layers=18, batch_size=1, image_shape=image_shape)
+print(relay_mod.astext(show_meta_data=False))
+
+
+###############################################################################
+# Create PassContext With Instruments
+# -----------------------------------
+# It is as simple as passing ``instruments`` argument to ``PassContext`` constructor.
+# A built-in ``PassTimingInstrument`` is used to profile the execution time of
+# each passes.
+timing_inst = PassTimingInstrument()
+with tvm.transform.PassContext(instruments=[timing_inst]):
+    relay_mod = relay.transform.InferType()(relay_mod)
+    relay_mod = relay.transform.FoldScaleAxis()(relay_mod)
+    # before exiting the context, get profile results.
+    profiles = timing_inst.render()
+print(profiles)
+
+
+###############################################################################
+# One can also use the current ``PassContext`` and register
+# ``PassInstrument`` instances by ``override_instruments`` method.
+# Note that ``override_instruments`` executes ``exit_pass_ctx`` callbacks
+# if any instrument already exists. Then it switches to new instruments
+# and calls ``enter_pass_ctx`` callbacks of new instruments.
+# Refer to following sections and :py:func:`tvm.instrument.pass_instrument` for these callbacks.
+cur_pass_ctx = tvm.transform.PassContext.current()
+cur_pass_ctx.override_instruments([timing_inst])
+relay_mod = relay.transform.InferType()(relay_mod)
+relay_mod = relay.transform.FoldScaleAxis()(relay_mod)
+profiles = timing_inst.render()
+print(profiles)
+
+
+###############################################################################
+# Register empty list to clear instruments.
+#
+# Note that ``exit_pass_ctx`` of ``PassTimingInstrument`` is called.
+# Profiles are cleared so nothing is printed.
+cur_pass_ctx.override_instruments([])
+# Uncomment the call to .render() to see a warning like:
+# Warning: no passes have been profiled, did you enable pass profiling?
+# profiles = timing_inst.render()
+
+
+###############################################################################
+# Create Customized Instrument Class
+# ----------------------------------
+# A customized instrument class can be easily created by
+# :py:func:`tvm.instrument.pass_instrument` decorator.
+#
+# Let's create an instrument class which calculate the difference of ``CallNode``
+# counting per ``op.name`` before and after passes.
+
+# decorate the class
+@pass_instrument
+class RelayCallNodeDiffer:
+    def __init__(self):
+        self._op_diff = []
+        # Passes can be nested.
+        # Use stack to make sure we get correct before/after pairs.
+        self._op_cnt_before_stack = []
+
+    def enter_pass_ctx(self):
+        self._op_diff = []
+        self._op_cnt_before_stack = []
+
+    def exit_pass_ctx(self):
+        assert len(self._op_cnt_before_stack) == 0, "The stack is not empty. Something wrong."
+
+    def run_before_pass(self, mod, info):
+        self._op_cnt_before_stack.append((info.name, self._count_nodes(mod)))
+
+    def run_after_pass(self, mod, info):
+        # Pop out the latest recorded pass.
+        name_before, op_to_cnt_before = self._op_cnt_before_stack.pop()
+        assert name_before == info.name, "name_before: {}, info.name: {} doesn't match".format(
+            name_before, info.name
+        )
+        cur_depth = len(self._op_cnt_before_stack)
+        op_to_cnt_after = self._count_nodes(mod)
+        op_diff = self._diff(op_to_cnt_after, op_to_cnt_before)
+        # only record passes causing differences.
+        if op_diff:
+            self._op_diff.append((cur_depth, info.name, op_diff))
+
+    def get_pass_to_op_diff(self):
+        """
+        return [
+          (depth, pass_name, {op_name: diff_num, ...}), ...
+        ]
+        """
+        return self._op_diff
+
+    @staticmethod
+    def _count_nodes(mod):
+        ret = {}
+
+        def visit(node):
+            if isinstance(node, relay.expr.Call):
+                try:
+                    op_name = node.op.name
+                except AttributeError:
+                    # Some CallNode may not have 'name' such as relay.Function
+                    return
+                try:
+                    ret[op_name] += 1
+                except KeyError:
+                    ret[op_name] = 1
+
+        relay.analysis.post_order_visit(mod["main"], visit)
+        return ret
+
+    @staticmethod
+    def _diff(d_after, d_before):
+        # d_after - d_before
+        ret = {}
+        key_after, key_before = set(d_after), set(d_before)
+        for k in key_before & key_after:
+            tmp = d_after[k] - d_before[k]
+            if tmp:
+                ret[k] = d_after[k] - d_before[k]
+        for k in key_after - key_before:
+            ret[k] = d_after[k]
+        for k in key_before - key_after:
+            ret[k] = -d_before[k]
+        return ret
+
+
+###############################################################################
+# Apply Passes and Multiple Instrument Classes
+# --------------------------------------------
+# Apply any pass you wish. Here :py:class:`tvm.relay.transform.ConvertLayout`
+# and :py:class:`tvm.relay.transform.FoldConstant` are used.
+#
+# ``ConvertLayout`` might add ``layout_transform`` Op while ``FoldConstant`` can
+# reduce the number of ``CallNode``.
+#
+# We can also use multiple instrument classes in a ``PassContext``.
+# However, it should be noted that instrument methods are executed sequentially,
+# obeying the order of ``instruments`` argument.
+# So for instrument classes like ``PassTimingInstrument``, it is inevitable to
+# count-up the execution time of other instrument classes to the final
+# profile result.
+call_node_inst = RelayCallNodeDiffer()
+desired_layouts = {
+    "nn.conv2d": ["NHWC", "HWIO"],
+}
+# Because layout_transform may be added as a successor of Constant,
+# we run FoldConstant twice.
+# Though it is obvious only the FoldConstant after the ConvertLayout matter,
+# we want to show how many layout_transform is added as a successor of
+# Constant.
+pass_seq = tvm.transform.Sequential(
+    [
+        relay.transform.FoldConstant(),
+        relay.transform.ConvertLayout(desired_layouts),
+        relay.transform.FoldConstant(),
+    ]
+)
+# bind parameters to make VarNode as ConstantNode.
+relay_mod["main"] = bind_params_by_name(relay_mod["main"], relay_params)
+# timing_inst is put after call_node_inst.
+# So the execution time of ``call_node.inst.run_after_pass()`` is also counted.
+with tvm.transform.PassContext(opt_level=3, instruments=[call_node_inst, timing_inst]):

Review comment:
       it would be super awesome to add another section demonstrating an exception case here!

##########
File path: docs/dev/pass_infra.rst
##########
@@ -389,6 +396,103 @@ To allow other C++ modules to apply this pass, we declare a free function in
 
     TVM_DLL Pass FoldConstant();
 
+.. _pass_instrument_section_tag:
+
+Pass Instrument
+~~~~~~~~~~~~~~~
+
+``PassInstrument`` provides callbacks run when entering/exiting ``PassContext`` and before/after executing passes.
+Multiple ``PassInstrument`` instances can be registed into a single ``PassContext``.
+Instrument instances are called sequentially in the order of ``instruments`` argument passed to ``PassContext``.
+
+.. code:: c++
+
+    namespace instrument {
+
+    class PassInstrumentNode : public Object {
+     public:
+      String name;
+      virtual void EnterPassContext() const = 0;
+      virtual void ExitPassContext() const = 0;
+      virtual bool ShouldRun(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      virtual void RunBeforePass(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      virtual void RunAfterPass(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      /* Other fields are omitted. */
+    };
+
+    class PassInstrument : public ObjectRef {
+     public:
+      TVM_DEFINE_OBJECT_REF_METHODS(PassInstrument, ObjectRef, PassInstrumentNode);
+    };
+
+    }  // namespace instrument
+
+Python interfaces are provided to implement ``PassInstrument`` quickly.

Review comment:
       Perhaps it would be good to include a pointer to these, since a pointer to the C++ ones are above.

##########
File path: docs/dev/pass_infra.rst
##########
@@ -389,6 +396,103 @@ To allow other C++ modules to apply this pass, we declare a free function in
 
     TVM_DLL Pass FoldConstant();
 
+.. _pass_instrument_section_tag:
+
+Pass Instrument
+~~~~~~~~~~~~~~~
+
+``PassInstrument`` provides callbacks run when entering/exiting ``PassContext`` and before/after executing passes.
+Multiple ``PassInstrument`` instances can be registed into a single ``PassContext``.
+Instrument instances are called sequentially in the order of ``instruments`` argument passed to ``PassContext``.
+
+.. code:: c++
+
+    namespace instrument {
+
+    class PassInstrumentNode : public Object {
+     public:
+      String name;
+      virtual void EnterPassContext() const = 0;
+      virtual void ExitPassContext() const = 0;
+      virtual bool ShouldRun(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      virtual void RunBeforePass(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      virtual void RunAfterPass(const IRModule& mod, const transform::PassInfo& info) const = 0;
+      /* Other fields are omitted. */
+    };
+
+    class PassInstrument : public ObjectRef {
+     public:
+      TVM_DEFINE_OBJECT_REF_METHODS(PassInstrument, ObjectRef, PassInstrumentNode);
+    };
+
+    }  // namespace instrument
+
+Python interfaces are provided to implement ``PassInstrument`` quickly.
+
+Following four methods are invoked in the life-cycle of ``PassContext``.
+
+.. code:: c++
+
+    TVM_DLL void InstrumentEnterPassContext();
+    TVM_DLL void InstrumentExitPassContext();
+    TVM_DLL bool InstrumentBeforePass(const IRModule& mod, const PassInfo& info) const;
+    TVM_DLL void InstrumentAfterPass(const IRModule& mod, const PassInfo& info) const;
+
+``InstrumentEnterPassContext`` is called immediately when the scope
+of the ``PassContext`` instance is entered.
+
+``InstrumentExitPassContext`` is called when the scope of ``PassContextNode``
+is being leaved, or exceptions occur during the execution of passes.
+This method is also called when instruments is being overriden by ``override_instruments`` in ::py:class:`tvm.transform.PassContext`.
+
+``InstrumentBeforePass`` is called before pass-execution.
+``InstrumentAfterPass`` is called after pass-executioon if the pass should be run. The behavir is like:
+
+.. code:: c++
+
+      if (pass_ctx.InstrumentBeforePass(ir_module, pass_info)) {
+        new_ir_module = run_pass(ir_module, pass_ctx);
+        pass_ctx.InstrumentAfterPass(new_ir_module, pass_info);
+        return new_ir_module;
+      }
+
+Here is a brief introduction of each methods. See (`src/ir/transform.cc`_) for more details.
+
+- ``InstrumentEnterPassContext``
+
+  * ``EnterPassContext()`` is executed in the order of ``instruments`` passed to the ``PassContext``.
+  * When an exception raises, ``PassContext`` disable the pass instrumentation
+    by clearing all registered ``PassInstrument`` instances.
+  * Then ``PassContext`` execute ``ExitPassContext()`` method of each ``PassInstrument``
+    instances which successfully finished ``EnterPassContext()``
+  * For example, if ``PassInstrument`` A, B, and C are registered to a ``PassContext``
+    and A finished ``EnterPassContext()`` while B throws an exception, then C
+    is never executed; ``ExitPassContext()`` of A is executed.
+
+- ``InstrumentExitPassContext``
+
+  * ``ExitPassContext()`` of each ``PassInstrument`` instances are executed in
+    the order of ``instruments`` passed to the ``PassContext``.
+  * While an exception occurs, ``instruments`` is cleared.
+  * That means, instances registered after the one throwing exceptions do not execute ``ExitPassContext``.
+
+- ``InstrumentBeforePass``
+
+  * ``ShouldRun`` callbakc is executed if the pass is not listed as a required pass.

Review comment:
       i think you could just delete the word callback here. ShouldRun is an interface method.

##########
File path: docs/dev/pass_infra.rst
##########
@@ -389,6 +396,103 @@ To allow other C++ modules to apply this pass, we declare a free function in
 
     TVM_DLL Pass FoldConstant();
 
+.. _pass_instrument_section_tag:
+
+Pass Instrument
+~~~~~~~~~~~~~~~
+
+``PassInstrument`` provides callbacks run when entering/exiting ``PassContext`` and before/after executing passes.

Review comment:
       Since PassInstrument is an interface, it might be more helpful to speak about the interface as a whole rather than mention callbacks, which are just one way of implementing the interface. e.g.
   
   The PassInstrument interface allows you to run code when entering/exiting PassContext and before ...




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