comaniac commented on a change in pull request #8702:
URL: https://github.com/apache/tvm/pull/8702#discussion_r701467431



##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration

Review comment:
       Update docstring

##########
File path: tests/python/relay/test_pipeline_executor.py
##########
@@ -0,0 +1,276 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+import numpy as np
+import tvm
+import tvm.testing
+from tvm import relay
+from tvm.relay import transform
+from tvm.contrib import graph_executor, pipeline_executor
+
+
+def get_mannual_mod():
+    """
+    # get list of module that represent a subgraph
+    """
+    mods = []
+    dshape = (3, 3)
+    data = relay.var("data_0", relay.TensorType(dshape, "float32"))
+    data21 = relay.var("data_1", relay.TensorType(dshape, "float32"))
+    data_net1_output_1 = relay.var("data_0", relay.TensorType(dshape, 
"float32"))
+    data_net1_output_2 = relay.var("data_1", relay.TensorType(dshape, 
"float32"))
+    data_net2_output_1 = relay.var("data_0", relay.TensorType(dshape, 
"float32"))
+    mvalue1 = np.full((1), 1).astype("float32")
+    mvalue2 = np.full((1), 2).astype("float32")
+    mvalue3 = np.full((1), 3).astype("float32")
+    mv1 = relay.Constant(tvm.nd.array(mvalue1))
+    mv2 = relay.Constant(tvm.nd.array(mvalue2))
+    mv3 = relay.Constant(tvm.nd.array(mvalue3))
+
+    """
+    # net1 have three output, output3 is final output.
+    """
+
+    net_output1 = relay.add(data, mv1)
+    net_output2 = relay.subtract(data, mv2)
+    net_output3 = relay.multiply(data, mv3)
+
+    """
+    # net2 use net1 output1 as input.
+    """
+    net2 = relay.add(data_net1_output_1, mv2)
+    net2 = relay.add(net2, data21)
+    net2 = relay.add(net2, mv3)
+
+    """
+    # net3 use net2 output1 and net1 outpu2 as input.
+    """
+    net3 = relay.multiply(data_net2_output_1, mv3)
+    net3 = relay.add(net3, data_net1_output_2)
+
+    mods.append(
+        tvm.IRModule.from_expr(
+            relay.Function([data], relay.Tuple([net_output1, net_output2, 
net_output3]))
+        )
+    )
+    mods.append(tvm.IRModule.from_expr(relay.Function([data_net1_output_1, 
data21], net2)))
+    mods.append(
+        tvm.IRModule.from_expr(relay.Function([data_net1_output_2, 
data_net2_output_1], net3))
+    )
+
+    return mods, dshape
+
+
+def get_manual_conf(mods, target):
+    """
+    # This function use to generate manual pipe line configueration,
+    # the result use to verify if the pipe configuration can generate
+    # correct result.
+    """
+    mod_config = {}
+    """
+    # set configure
+    """
+
+    """
+    # third output is final output, second output for mod3, first for mod2
+    # input
+    """
+    pipe_config1 = {
+        "mod_indx": 1,
+        "output": [
+            {"output_indx": 0, "dependent": [{"mod_indx": 2, "input_name": 
"data_0"}]},
+            {"output_indx": 1, "dependent": [{"mod_indx": 3, "input_name": 
"data_0"}]},
+            {"output_indx": 2, "dependent": [{"mod_indx": 0, "input_name": 
"0"}]},
+        ],
+    }
+    mod_config[mods[0]] = {
+        "pipeline": pipe_config1,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": target[0],
+        "dev": target[1],
+    }
+
+    pipe_config2 = {
+        "mod_indx": 2,
+        "output": [
+            {"output_indx": 0, "dependent": [{"mod_indx": 3, "input_name": 
"data_1"}]},
+        ],
+    }
+    mod_config[mods[1]] = {
+        "pipeline": pipe_config2,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": "llvm",
+        "dev": tvm.cpu(0),
+    }
+
+    pipe_config3 = {
+        "mod_indx": 3,
+        "output": [{"output_indx": 0, "dependent": [{"mod_indx": 0, 
"input_name": "1"}]}],
+    }
+    mod_config[mods[2]] = {
+        "pipeline": pipe_config3,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": "llvm",
+        "dev": tvm.cpu(0),
+    }
+    return mod_config
+
+
+def pipeline(target):

Review comment:
       You may consider inline this function directly to `test_pipeline` as it 
is only used by one function.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():

Review comment:
       ```suggestion
                   for binding in inputs.values():
   ```

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)

Review comment:
       This shouldn't be a "config". It's weird to see something like 
`mod_config = build(pipe_config)`.
   This should derive from `ExecutorFactoryModule` like 
`GraphExecutorFactoryModule`, so maybe call it `PipelineExecutorFactoryModule`.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.

Review comment:
       ```suggestion
               """The class that use to storage module connection information.
               The binding can be either "input" or "output".
   
               Parameters
               ----------
               owner : ModuleWrapper
                   The class that own this interface, in such class there are
                   Module information like index, module name
   
               io_type : str
                   The type of this binding. It can be either "input" or 
"output".
   
               name : str/integer
                   Binding name, for input it is string such as "data0";
                   for output it is the index integer such as 0.
   ```

##########
File path: tests/python/relay/test_pipeline_executor.py
##########
@@ -0,0 +1,276 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+import numpy as np
+import tvm
+import tvm.testing
+from tvm import relay
+from tvm.relay import transform
+from tvm.contrib import graph_executor, pipeline_executor
+
+
+def get_mannual_mod():
+    """
+    # get list of module that represent a subgraph
+    """
+    mods = []
+    dshape = (3, 3)
+    data = relay.var("data_0", relay.TensorType(dshape, "float32"))
+    data21 = relay.var("data_1", relay.TensorType(dshape, "float32"))
+    data_net1_output_1 = relay.var("data_0", relay.TensorType(dshape, 
"float32"))
+    data_net1_output_2 = relay.var("data_1", relay.TensorType(dshape, 
"float32"))
+    data_net2_output_1 = relay.var("data_0", relay.TensorType(dshape, 
"float32"))
+    mvalue1 = np.full((1), 1).astype("float32")
+    mvalue2 = np.full((1), 2).astype("float32")
+    mvalue3 = np.full((1), 3).astype("float32")
+    mv1 = relay.Constant(tvm.nd.array(mvalue1))
+    mv2 = relay.Constant(tvm.nd.array(mvalue2))
+    mv3 = relay.Constant(tvm.nd.array(mvalue3))
+
+    """
+    # net1 have three output, output3 is final output.
+    """
+
+    net_output1 = relay.add(data, mv1)
+    net_output2 = relay.subtract(data, mv2)
+    net_output3 = relay.multiply(data, mv3)
+
+    """
+    # net2 use net1 output1 as input.
+    """
+    net2 = relay.add(data_net1_output_1, mv2)
+    net2 = relay.add(net2, data21)
+    net2 = relay.add(net2, mv3)
+
+    """
+    # net3 use net2 output1 and net1 outpu2 as input.
+    """
+    net3 = relay.multiply(data_net2_output_1, mv3)
+    net3 = relay.add(net3, data_net1_output_2)
+
+    mods.append(
+        tvm.IRModule.from_expr(
+            relay.Function([data], relay.Tuple([net_output1, net_output2, 
net_output3]))
+        )
+    )
+    mods.append(tvm.IRModule.from_expr(relay.Function([data_net1_output_1, 
data21], net2)))
+    mods.append(
+        tvm.IRModule.from_expr(relay.Function([data_net1_output_2, 
data_net2_output_1], net3))
+    )
+
+    return mods, dshape
+
+
+def get_manual_conf(mods, target):
+    """
+    # This function use to generate manual pipe line configueration,
+    # the result use to verify if the pipe configuration can generate
+    # correct result.
+    """
+    mod_config = {}
+    """
+    # set configure
+    """
+
+    """
+    # third output is final output, second output for mod3, first for mod2
+    # input
+    """
+    pipe_config1 = {
+        "mod_indx": 1,
+        "output": [
+            {"output_indx": 0, "dependent": [{"mod_indx": 2, "input_name": 
"data_0"}]},
+            {"output_indx": 1, "dependent": [{"mod_indx": 3, "input_name": 
"data_0"}]},
+            {"output_indx": 2, "dependent": [{"mod_indx": 0, "input_name": 
"0"}]},
+        ],
+    }
+    mod_config[mods[0]] = {
+        "pipeline": pipe_config1,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": target[0],
+        "dev": target[1],
+    }
+
+    pipe_config2 = {
+        "mod_indx": 2,
+        "output": [
+            {"output_indx": 0, "dependent": [{"mod_indx": 3, "input_name": 
"data_1"}]},
+        ],
+    }
+    mod_config[mods[1]] = {
+        "pipeline": pipe_config2,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": "llvm",
+        "dev": tvm.cpu(0),
+    }
+
+    pipe_config3 = {
+        "mod_indx": 3,
+        "output": [{"output_indx": 0, "dependent": [{"mod_indx": 0, 
"input_name": "1"}]}],
+    }
+    mod_config[mods[2]] = {
+        "pipeline": pipe_config3,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": "llvm",
+        "dev": tvm.cpu(0),
+    }
+    return mod_config
+
+
+def pipeline(target):
+    """
+    #Get 3 pipeline module.
+    """
+    (mod1, mod2, mod3), dshape = get_mannual_mod()
+
+    # Prepare batch data for pipeline feeding
+    datas = []
+    for i in range(5):
+        datas.append(np.full(dshape, 3 + i).astype("float32"))
+
+    # Runtime error check
+    pipe_config_check(mod1, mod2, mod3)

Review comment:
       Put this to a separate test. For example, you can have
   ```python
   def test_pipeline_config():
         (mod1, mod2, mod3), dshape = get_mannual_mod()
         # Inline "pipe_config_check" here
   ```
   
   This check should be target independent so it should not be tested many 
times. Putting it to a separate test can also help locate the problem when test 
failure.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():
+                    if start == binding.io_owner:
+                        return False
+                    for p in binding.parents:
+                        if not self.dag_acircle_check(start, 
p.io_owner.input_bindings.bindings):
+                            return False
+
+                return True
+
+            def connect(self, binding):
+                """
+                # check if the bindendency setting correct.
+                # correct connection are following
+                # 1. global input to module input
+                # 2. module output to global output
+                # 3. module output to moudle input
+                """
+                owner_indx = self.get_owner_indx()
+                bind_owner_indx = binding.get_owner_indx()
+                if owner_indx == bind_owner_indx:
+                    raise RuntimeError(f"can not set self as binding.")

Review comment:
       By getting rid of the index, you should implement this check like
   ```
   if self.io_owner == binding.io_owner:
       raise RuntimeError(f"Can not set self as binding.")
   ```
   
   Then you need to implement `ModuleWrapper.__eq__(self, other)` to support 
the `==` operator.
   
   

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():
+                    if start == binding.io_owner:
+                        return False
+                    for p in binding.parents:
+                        if not self.dag_acircle_check(start, 
p.io_owner.input_bindings.bindings):
+                            return False
+
+                return True
+
+            def connect(self, binding):
+                """
+                # check if the bindendency setting correct.
+                # correct connection are following
+                # 1. global input to module input
+                # 2. module output to global output
+                # 3. module output to moudle input
+                """
+                owner_indx = self.get_owner_indx()
+                bind_owner_indx = binding.get_owner_indx()
+                if owner_indx == bind_owner_indx:
+                    raise RuntimeError(f"can not set self as binding.")
+
+                if owner_indx != 0 and self.io_type == "input":
+                    raise RuntimeError(f"Module only can start binding from 
output!")
+
+                if owner_indx != 0 and bind_owner_indx != 0 and 
binding.io_type == "output":
+                    raise RuntimeError(f"Module output can not binding with 
module output!")
+
+                if owner_indx != 0 and bind_owner_indx == 0 and 
binding.io_type == "input":
+                    raise RuntimeError(f"Module output can not binding with 
global input!")
+
+                if owner_indx == 0 and self.io_type != "input":
+                    raise RuntimeError(f"Global only can start binding from 
input!")
+
+                if owner_indx == 0 and binding.io_type != "input":
+                    raise RuntimeError(f"Global input only can set binding 
with module input.")
+
+                self.bindings.append(binding)
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):

Review comment:
       This can also be `if not self.is_global_interface()`

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):

Review comment:
       The word "acircle" seems not that common. Maybe just "check_dag_acyclic" 
and return True if it is valid (no circle)?

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():
+                    if start == binding.io_owner:
+                        return False
+                    for p in binding.parents:
+                        if not self.dag_acircle_check(start, 
p.io_owner.input_bindings.bindings):
+                            return False
+
+                return True
+
+            def connect(self, binding):
+                """
+                # check if the bindendency setting correct.
+                # correct connection are following
+                # 1. global input to module input
+                # 2. module output to global output
+                # 3. module output to moudle input
+                """
+                owner_indx = self.get_owner_indx()
+                bind_owner_indx = binding.get_owner_indx()
+                if owner_indx == bind_owner_indx:
+                    raise RuntimeError(f"can not set self as binding.")
+
+                if owner_indx != 0 and self.io_type == "input":

Review comment:
       Again, all `owner_idx != 0` can just be `not self.is_global_interface()`.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0

Review comment:
       I still feel use 0 to indicate global interface is not clear enough. I 
read through the use cases, you only use it for `index != 0`, so it should be 
fine to just make this function to be
   
   ```
   def is_global_interface(self):
       return not isinstance(self.io_owner, PipelineConfig.ModuleWrapper)
   ```
   
   btw, please change all `xx_indx`  to `xx_index` or `xx_idx`.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():
+                    if start == binding.io_owner:
+                        return False
+                    for p in binding.parents:
+                        if not self.dag_acircle_check(start, 
p.io_owner.input_bindings.bindings):
+                            return False
+
+                return True
+
+            def connect(self, binding):
+                """
+                # check if the bindendency setting correct.
+                # correct connection are following
+                # 1. global input to module input
+                # 2. module output to global output
+                # 3. module output to moudle input
+                """
+                owner_indx = self.get_owner_indx()
+                bind_owner_indx = binding.get_owner_indx()
+                if owner_indx == bind_owner_indx:
+                    raise RuntimeError(f"can not set self as binding.")
+
+                if owner_indx != 0 and self.io_type == "input":
+                    raise RuntimeError(f"Module only can start binding from 
output!")
+
+                if owner_indx != 0 and bind_owner_indx != 0 and 
binding.io_type == "output":
+                    raise RuntimeError(f"Module output can not binding with 
module output!")
+
+                if owner_indx != 0 and bind_owner_indx == 0 and 
binding.io_type == "input":
+                    raise RuntimeError(f"Module output can not binding with 
global input!")
+
+                if owner_indx == 0 and self.io_type != "input":
+                    raise RuntimeError(f"Global only can start binding from 
input!")
+
+                if owner_indx == 0 and binding.io_type != "input":
+                    raise RuntimeError(f"Global input only can set binding 
with module input.")
+
+                self.bindings.append(binding)
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    # check if the source and target data_type same
+                    if (
+                        isinstance(binding.io_owner, 
PipelineConfig.ModuleWrapper)
+                        and self.data_type != binding.data_type
+                    ):
+                        raise RuntimeError(f"Illegal type:binding type is not 
same!")
+
+                    binding.parents.append(self)
+                    # Do acircle check after add the in-degree.
+                    if not self.dag_acircle_check(
+                        binding.io_owner, self.io_owner.input_bindings.bindings
+                    ):
+                        raise RuntimeError(f"Illegal connection: cause a 
circle!")
+
+        def __init__(self, mod=None, index=0):
+            """init class"""
+            self.input_params = InferType()(mod)["main"].params
+            self.output_values = InferType()(mod)["main"].checked_type.ret_type
+            self.set_index_name(index)
+            self.mod = mod
+            self.input_bindings = PipelineConfig.BindingList(self, "input")
+            self.output_bindings = PipelineConfig.BindingList(self, "output")
+            self.target_host_ = None
+            self.build_func_ = None
+            self.params_ = None
+            self.target_ = None
+            self.dev_ = None
+
+        def __getitem__(self, key):
+            """get item by key"""
+            if isinstance(key, str):
+                if key == "input":
+                    return self.input_bindings
+
+                if key == "output":
+                    return self.output_bindings
+
+            raise RuntimeError(f"{key} not found!")
+
+        def get_data_type(self, key, stype):
+            """get module input/output data type."""
+
+            if stype == "input":
+                for param in self.input_params:
+                    if param.name_hint == key:
+                        return param._checked_type_
+
+            if stype == "output":
+                if isinstance(self.output_values, tvm.ir.type.TupleType):
+                    if int(key) < len(self.output_values.fields):
+                        return self.output_values.fields[int(key)]
+                elif int(key) == 0:
+                    return self.output_values
+
+            return None
+
+        def set_index_name(self, index):
+            """generate name by index and storage index value"""
+            self.index = index
+            self.name = "mod{}".format(str(index))
+
+        def is_root_mod(self):
+            """use by DAG topology sort, identify if this item is root item 
and in-degree is 0"""
+            for _, binding in self.input_bindings.bindings.items():
+                if binding.parents:
+                    return False
+
+            return True
+
+        def remove_self_from_bindings(self):
+            """use by DAG topology sort, by remove self from binding to reduce 
child in-degree"""
+            for _, binding in self.output_bindings.bindings.items():
+                for child in binding.bindings:
+                    if binding in child.parents:
+                        child.parents.remove(binding)
+
+        def set_target_host(self, host):
+            """set target host that use by build function"""
+            self.target_host_ = host
+
+        def set_build_func(self, build_func):
+            """set build funciton that use by build function"""
+            self.build_func_ = build_func
+
+        def set_params(self, params):
+            """set params that use by build function"""
+            self.params_ = params
+
+        def set_target(self, target):
+            """set target that use by build function"""
+            self.target_ = target
+
+        def set_dev(self, dev):
+            """set dev that use by build function"""
+            self.dev_ = dev
+
+    class BindingList:
+        """Use to storage Binding list.
+        Parameters
+        ----------
+
+        owner : ModuleWrapper/PipelineConfig
+            who own this list, it can be ModuleWrapper or PipelineConfig
+
+        type_name : str
+            The type of this binding list. It can be either "input" or 
"output".
+        """
+
+        def __init__(self, owner, type_name):
+            self.bindings = {}
+            self.io_owner = owner
+            self.binding_type = type_name
+
+        def get_binding_data_type(self, key):
+            """return binding data type"""
+            if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                return self.io_owner.get_data_type(key, self.binding_type)
+
+            return None
+
+        def __getitem__(self, key):
+            """return item by key"""
+            if key not in self.bindings:
+                data_type = self.get_binding_data_type(key)
+                if not data_type and isinstance(self.io_owner, 
PipelineConfig.ModuleWrapper):
+                    raise RuntimeError(f"Illegal name: 
{self.binding_type}:{key} cannot find")
+
+                self.bindings[key] = PipelineConfig.ModuleWrapper.Binding(
+                    self.io_owner, self.binding_type, key, data_type
+                )
+
+            return self.bindings[key]
+
+    def __init__(self):
+        self.last_mod_indx = 0
+        self.mod_wrapper = {}
+        self.input_bindings = self.BindingList(self, "input")
+        self.output_bindings = self.BindingList(self, "output")
+
+    def __str__(self):
+        """ Get configuration in string type"""
+        # Sort moudles
+        self.dag_topology_sort()
+
+        # get input
+        input_dump = "Inputs\n"
+        for input_name in self.input_bindings.bindings:
+            inf = self.input_bindings.bindings[input_name]
+            input_dump += inf.__repr__() + "\n"
+
+        # get connections
+        output = {}
+        connections_dump = "\nconnections\n"
+        for mod in self.mod_wrapper:
+            for _, interface in 
self.mod_wrapper[mod].output_bindings.bindings.items():
+                if interface.bindings:
+                    mname, dname = interface.get_name()
+                    iname = mname + ".output(" + dname + ")->"
+                    for dep in interface.bindings:
+                        dep_mname, dep_dname = dep.get_name()
+                        if isinstance(dep.io_owner, 
PipelineConfig.ModuleWrapper):
+                            iname += " " + dep_mname + "." + dep_dname
+                            connections_dump += "  |" + iname + "\n"
+                        else:
+                            output[dep_dname] = mname + ".output(" + dname + 
")"
+
+        # get output
+        output_dump = "\noutput\n"
+        for name in sorted(output.keys()):
+            output_dump += "  |output(" + name + ") : " + output[name] + "\n"
+
+        return input_dump + output_dump + connections_dump
+
+    def __getitem__(self, key):
+        """return item by key"""
+        if isinstance(key, tvm.ir.module.IRModule):
+            if key not in self.mod_wrapper:
+                # self.last_mod_indx start from 1 and be initialize value,
+                # the final value for mod index would get generate by function
+                # dag_topology_sort

Review comment:
       In this case can we just let index unset until topological sort? It's 
weird to see module index has two meanings (adding order and topological order).

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():
+                    if start == binding.io_owner:
+                        return False
+                    for p in binding.parents:
+                        if not self.dag_acircle_check(start, 
p.io_owner.input_bindings.bindings):
+                            return False
+
+                return True
+
+            def connect(self, binding):
+                """
+                # check if the bindendency setting correct.
+                # correct connection are following
+                # 1. global input to module input
+                # 2. module output to global output
+                # 3. module output to moudle input
+                """
+                owner_indx = self.get_owner_indx()
+                bind_owner_indx = binding.get_owner_indx()
+                if owner_indx == bind_owner_indx:
+                    raise RuntimeError(f"can not set self as binding.")
+
+                if owner_indx != 0 and self.io_type == "input":
+                    raise RuntimeError(f"Module only can start binding from 
output!")
+
+                if owner_indx != 0 and bind_owner_indx != 0 and 
binding.io_type == "output":
+                    raise RuntimeError(f"Module output can not binding with 
module output!")
+
+                if owner_indx != 0 and bind_owner_indx == 0 and 
binding.io_type == "input":
+                    raise RuntimeError(f"Module output can not binding with 
global input!")
+
+                if owner_indx == 0 and self.io_type != "input":
+                    raise RuntimeError(f"Global only can start binding from 
input!")
+
+                if owner_indx == 0 and binding.io_type != "input":
+                    raise RuntimeError(f"Global input only can set binding 
with module input.")
+
+                self.bindings.append(binding)
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    # check if the source and target data_type same
+                    if (
+                        isinstance(binding.io_owner, 
PipelineConfig.ModuleWrapper)
+                        and self.data_type != binding.data_type
+                    ):
+                        raise RuntimeError(f"Illegal type:binding type is not 
same!")

Review comment:
       ```suggestion
                           raise RuntimeError(f"Illegal type (%s vs. %s): 
binding type is not same!" % (self.data_type, binding.data_type))
   ```

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -0,0 +1,559 @@
+# 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.
+"""Pipeline executor that executes a series of modules in a pipeline 
fashion."""
+import json
+import tvm._ffi
+from tvm import relay
+from tvm.relay.transform import InferType
+from tvm.contrib import graph_executor
+
+
+def pipeline_executor_enabled():
+    """check if pipeline executor is enabled.
+
+    Return
+    -------
+    enable: bool
+        Return pipeline executor is enabled or not.
+    """
+    return tvm._ffi.get_global_func("tvm.pipeline_executor.create", 
allow_missing=True) is not None
+
+
+def build(pipe_configs):
+    """build module list that can use for pipeline execution.
+
+    Parameters
+    ----------
+    mod_n_configs: Dict[IRModule, Dict[str, Any]]
+        build configuration informaton, structure like following.
+        {IRModule: {"target":target,
+                    "target_host":target_host,
+                    "params":params,
+                    "mod_name"mod_name,
+                    "build":build}}
+
+    Returns
+    -------
+    ret: List[IRModule]
+        list of IRModule
+    string_config: Dict[int, Dict[str, any]]
+        pipeline configuration
+    """
+    mods = {}
+    mod_n_configs = pipe_configs.get_config()
+    config_len = len(mod_n_configs)
+    string_config = [{} for _ in range(config_len)]
+    for ir_mod, mod_config in mod_n_configs.items():
+        mconf = mod_config["pipeline"].copy()
+        mod_indx = mconf["mod_indx"] - 1
+        # Get mod device config
+        dev = mod_config["dev"]
+        target = mod_config["target"]
+        build_func = relay.build
+        # if there is a self defined build function then use it.
+        if "build" in mod_config and mod_config["build"]:
+            build_func = mod_config["build"]
+
+        # build IRModule
+        mod = build_func(
+            ir_mod,
+            target,
+            params=mod_config["params"],
+            target_host=mod_config["target_host"],
+            mod_name=mod_config["mod_name"],
+        )
+
+        mconf["dev"] = "{},{}".format(dev.device_type, dev.device_id)
+        # Create pipeline configuration
+        string_config[mod_indx] = mconf
+        # associate mod with device
+        mods[mod] = {"dev": dev}
+
+    # return PipeModuleConfig
+    return PipeModuleConfig(mods, string_config)
+
+
+def create(pipe_mod_config):
+    """Create a pipeline runtime executor.
+
+    Parameters
+    ----------
+
+    pipe_mod_config : PipeModuleConfig
+        class to storage IRModule list and pipeline configuration.
+    -------
+
+    Returns
+    submodule : PipelineModule
+        Runtime pipeline module.
+    """
+
+    return PipelineModule(pipe_mod_config)
+
+
+class PipelineModule(object):
+    """Wrapper runtime module. This is a thin wrapper of the underlying TVM 
module.
+
+    Parameters
+    ----------
+    pipeline_mods : List[GraphModule]
+        The internal tvm module that holds the actual graph functions.
+    pipeline_config : Dict[IRModule, Dict[str, Any]]
+        modules and modules dependency configuration informaiton.
+    """
+
+    def __init__(self, pipe_mod_config):
+        self.pipeline_mods_ = pipe_mod_config.pipeline_mods_
+        self.mod_config_ = pipe_mod_config.mods_config_
+        mods, config = self.graph_executor_create(self.pipeline_mods_, 
self.mod_config_)
+        assert (
+            pipeline_executor_enabled()
+        ), "Pipeline executor is not enabled. Please \
+              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
+        pipelinecreate = tvm._ffi.get_global_func(
+            "tvm.pipeline_executor.create", allow_missing=False
+        )
+        assert pipelinecreate
+        module = pipelinecreate(mods, config)
+
+        self.module_ = module
+
+    def graph_executor_create(self, pipeline_mods, mod_config):
+        """Create graph_executor list and return string format config.
+
+        Parameters
+        ----------
+
+        pipeline_mods : List[IRModule]
+          list of IRModule
+
+        mod_config : Dict[int, Dict[str, Any]]
+            modules and modules dependency configuration informaiton.
+
+        Returns
+        -------
+        mods : List[GraphModule]
+            Runtime graph module.
+
+        mod_config : str
+            mods configuration
+        """
+
+        mods = []
+        for pipeline_mod in pipeline_mods:
+            mod = graph_executor.GraphModule(
+                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
+            )
+            mods.append(mod.module)
+
+        return mods, json.dumps(mod_config)
+
+
+class PipelineConfig(object):
+    """The wrapper of each module to be pipelined. The wrapper mainly includes 
the
+    module itself as well as the binding that represents the connections of 
this
+    module's inputs and outputs to other modules.
+    """
+
+    class ModuleWrapper:
+        """The class use use to represent Module and storage module index and
+        Binding information.
+        """
+
+        class Binding:
+            """The class that use to storage module connection information.
+               There are 2 types Binding Input:1 Output:2
+            Parameters
+            ----------
+
+            owner : ModuleWrapper
+                The class that own this interface, in such class there are
+                Module information like index, module name
+
+            io_type : str
+                The type of this binding. It can be either "input" or "output".
+
+            name : str/integer
+                Binding name, for input that is string for example "data0"
+                for output that is integer for example 0.
+            """
+
+            def __init__(self, owner, stype, name, data_type=None):
+                self.io_owner = owner
+                self.io_type = stype
+                self.name = str(name)
+                # These item that have dependency relation with self
+                self.bindings = []
+                # The item that self depend
+                self.parents = []
+
+                self.data_type = data_type
+
+            def get_name(self):
+                """get owner name and self name"""
+                owner_name = ""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    owner_name = self.io_owner.name
+
+                return owner_name, self.name
+
+            def get_owner_indx(self):
+                """return index if owner is ModuleWrapper, if not return 0"""
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    return self.io_owner.index
+
+                # if not ModuleWrapper then owner is PipelineConfig, return 0
+                # to identify this is global interface
+                return 0
+
+            def __repr__(self):
+                """Get all binding(input data), exepect like |data_0: 
mod1:data_0"""
+                ret = "  |{}: ".format(self.name)
+                for binding in self.bindings:
+                    mname, dname = binding.get_name()
+                    ret += "{0}:{1} ".format(mname, dname)
+                return ret
+
+            def dag_acircle_check(self, start, inputs):
+                """check if the DAG that current binding stay is acircle"""
+                for _, binding in inputs.items():
+                    if start == binding.io_owner:
+                        return False
+                    for p in binding.parents:
+                        if not self.dag_acircle_check(start, 
p.io_owner.input_bindings.bindings):
+                            return False
+
+                return True
+
+            def connect(self, binding):
+                """
+                # check if the bindendency setting correct.
+                # correct connection are following
+                # 1. global input to module input
+                # 2. module output to global output
+                # 3. module output to moudle input
+                """
+                owner_indx = self.get_owner_indx()
+                bind_owner_indx = binding.get_owner_indx()
+                if owner_indx == bind_owner_indx:
+                    raise RuntimeError(f"can not set self as binding.")
+
+                if owner_indx != 0 and self.io_type == "input":
+                    raise RuntimeError(f"Module only can start binding from 
output!")
+
+                if owner_indx != 0 and bind_owner_indx != 0 and 
binding.io_type == "output":
+                    raise RuntimeError(f"Module output can not binding with 
module output!")
+
+                if owner_indx != 0 and bind_owner_indx == 0 and 
binding.io_type == "input":
+                    raise RuntimeError(f"Module output can not binding with 
global input!")
+
+                if owner_indx == 0 and self.io_type != "input":
+                    raise RuntimeError(f"Global only can start binding from 
input!")
+
+                if owner_indx == 0 and binding.io_type != "input":
+                    raise RuntimeError(f"Global input only can set binding 
with module input.")
+
+                self.bindings.append(binding)
+                if isinstance(self.io_owner, PipelineConfig.ModuleWrapper):
+                    # check if the source and target data_type same
+                    if (
+                        isinstance(binding.io_owner, 
PipelineConfig.ModuleWrapper)
+                        and self.data_type != binding.data_type
+                    ):
+                        raise RuntimeError(f"Illegal type:binding type is not 
same!")
+
+                    binding.parents.append(self)
+                    # Do acircle check after add the in-degree.
+                    if not self.dag_acircle_check(
+                        binding.io_owner, self.io_owner.input_bindings.bindings
+                    ):
+                        raise RuntimeError(f"Illegal connection: cause a 
circle!")
+
+        def __init__(self, mod=None, index=0):
+            """init class"""
+            self.input_params = InferType()(mod)["main"].params
+            self.output_values = InferType()(mod)["main"].checked_type.ret_type
+            self.set_index_name(index)
+            self.mod = mod
+            self.input_bindings = PipelineConfig.BindingList(self, "input")
+            self.output_bindings = PipelineConfig.BindingList(self, "output")
+            self.target_host_ = None
+            self.build_func_ = None
+            self.params_ = None
+            self.target_ = None
+            self.dev_ = None
+
+        def __getitem__(self, key):
+            """get item by key"""
+            if isinstance(key, str):
+                if key == "input":
+                    return self.input_bindings
+
+                if key == "output":
+                    return self.output_bindings
+
+            raise RuntimeError(f"{key} not found!")
+
+        def get_data_type(self, key, stype):
+            """get module input/output data type."""
+
+            if stype == "input":
+                for param in self.input_params:
+                    if param.name_hint == key:
+                        return param._checked_type_
+
+            if stype == "output":
+                if isinstance(self.output_values, tvm.ir.type.TupleType):
+                    if int(key) < len(self.output_values.fields):
+                        return self.output_values.fields[int(key)]
+                elif int(key) == 0:
+                    return self.output_values
+
+            return None
+
+        def set_index_name(self, index):
+            """generate name by index and storage index value"""
+            self.index = index
+            self.name = "mod{}".format(str(index))
+
+        def is_root_mod(self):
+            """use by DAG topology sort, identify if this item is root item 
and in-degree is 0"""
+            for _, binding in self.input_bindings.bindings.items():
+                if binding.parents:
+                    return False
+
+            return True
+
+        def remove_self_from_bindings(self):
+            """use by DAG topology sort, by remove self from binding to reduce 
child in-degree"""
+            for _, binding in self.output_bindings.bindings.items():
+                for child in binding.bindings:
+                    if binding in child.parents:
+                        child.parents.remove(binding)
+
+        def set_target_host(self, host):
+            """set target host that use by build function"""
+            self.target_host_ = host
+
+        def set_build_func(self, build_func):
+            """set build funciton that use by build function"""
+            self.build_func_ = build_func
+
+        def set_params(self, params):
+            """set params that use by build function"""
+            self.params_ = params
+
+        def set_target(self, target):
+            """set target that use by build function"""
+            self.target_ = target
+
+        def set_dev(self, dev):
+            """set dev that use by build function"""
+            self.dev_ = dev

Review comment:
       As I mentioned, Python doesn't have private member, so you can just name 
and access them normally. For example, `mod.set_target_host(host)` -> 
`mod.target_host = host`.

##########
File path: tests/python/relay/test_pipeline_executor.py
##########
@@ -0,0 +1,276 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+import numpy as np
+import tvm
+import tvm.testing
+from tvm import relay
+from tvm.relay import transform
+from tvm.contrib import graph_executor, pipeline_executor
+
+
+def get_mannual_mod():
+    """
+    # get list of module that represent a subgraph
+    """
+    mods = []
+    dshape = (3, 3)
+    data = relay.var("data_0", relay.TensorType(dshape, "float32"))
+    data21 = relay.var("data_1", relay.TensorType(dshape, "float32"))
+    data_net1_output_1 = relay.var("data_0", relay.TensorType(dshape, 
"float32"))
+    data_net1_output_2 = relay.var("data_1", relay.TensorType(dshape, 
"float32"))
+    data_net2_output_1 = relay.var("data_0", relay.TensorType(dshape, 
"float32"))
+    mvalue1 = np.full((1), 1).astype("float32")
+    mvalue2 = np.full((1), 2).astype("float32")
+    mvalue3 = np.full((1), 3).astype("float32")
+    mv1 = relay.Constant(tvm.nd.array(mvalue1))
+    mv2 = relay.Constant(tvm.nd.array(mvalue2))
+    mv3 = relay.Constant(tvm.nd.array(mvalue3))
+
+    """
+    # net1 have three output, output3 is final output.
+    """
+
+    net_output1 = relay.add(data, mv1)
+    net_output2 = relay.subtract(data, mv2)
+    net_output3 = relay.multiply(data, mv3)
+
+    """
+    # net2 use net1 output1 as input.
+    """
+    net2 = relay.add(data_net1_output_1, mv2)
+    net2 = relay.add(net2, data21)
+    net2 = relay.add(net2, mv3)
+
+    """
+    # net3 use net2 output1 and net1 outpu2 as input.
+    """
+    net3 = relay.multiply(data_net2_output_1, mv3)
+    net3 = relay.add(net3, data_net1_output_2)
+
+    mods.append(
+        tvm.IRModule.from_expr(
+            relay.Function([data], relay.Tuple([net_output1, net_output2, 
net_output3]))
+        )
+    )
+    mods.append(tvm.IRModule.from_expr(relay.Function([data_net1_output_1, 
data21], net2)))
+    mods.append(
+        tvm.IRModule.from_expr(relay.Function([data_net1_output_2, 
data_net2_output_1], net3))
+    )
+
+    return mods, dshape
+
+
+def get_manual_conf(mods, target):
+    """
+    # This function use to generate manual pipe line configueration,
+    # the result use to verify if the pipe configuration can generate
+    # correct result.
+    """
+    mod_config = {}
+    """
+    # set configure
+    """
+
+    """
+    # third output is final output, second output for mod3, first for mod2
+    # input
+    """
+    pipe_config1 = {
+        "mod_indx": 1,
+        "output": [
+            {"output_indx": 0, "dependent": [{"mod_indx": 2, "input_name": 
"data_0"}]},
+            {"output_indx": 1, "dependent": [{"mod_indx": 3, "input_name": 
"data_0"}]},
+            {"output_indx": 2, "dependent": [{"mod_indx": 0, "input_name": 
"0"}]},
+        ],
+    }
+    mod_config[mods[0]] = {
+        "pipeline": pipe_config1,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": target[0],
+        "dev": target[1],
+    }
+
+    pipe_config2 = {
+        "mod_indx": 2,
+        "output": [
+            {"output_indx": 0, "dependent": [{"mod_indx": 3, "input_name": 
"data_1"}]},
+        ],
+    }
+    mod_config[mods[1]] = {
+        "pipeline": pipe_config2,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": "llvm",
+        "dev": tvm.cpu(0),
+    }
+
+    pipe_config3 = {
+        "mod_indx": 3,
+        "output": [{"output_indx": 0, "dependent": [{"mod_indx": 0, 
"input_name": "1"}]}],
+    }
+    mod_config[mods[2]] = {
+        "pipeline": pipe_config3,
+        "target_host": None,
+        "mod_name": "default",
+        "build": None,
+        "params": None,
+        "target": "llvm",
+        "dev": tvm.cpu(0),
+    }
+    return mod_config
+
+
+def pipeline(target):
+    """
+    #Get 3 pipeline module.
+    """
+    (mod1, mod2, mod3), dshape = get_mannual_mod()
+
+    # Prepare batch data for pipeline feeding
+    datas = []
+    for i in range(5):
+        datas.append(np.full(dshape, 3 + i).astype("float32"))
+
+    # Runtime error check
+    pipe_config_check(mod1, mod2, mod3)
+
+    pipe_config = pipeline_executor.PipelineConfig()
+
+    # Create pipeline compute input/output and subgraph dependent relation.
+
+    # Test in key mode for binding find.
+    # pipeline compute input "data_0" would get forward to mod1 as input 
"data_0"
+    
pipe_config["input"]["data_0"].connect(pipe_config[mod1]["input"]["data_0"])
+
+    # pipeline compute input "data_1" would get forward to mod2 as input 
"data_1"
+    
pipe_config["input"]["data_1"].connect(pipe_config[mod2]["input"]["data_1"])
+
+    # mod1 output(0) would get forward to mod2 as input "data_0"
+    
pipe_config[mod1]["output"][0].connect(pipe_config[mod2]["input"]["data_0"])
+
+    # mod1 output(1) would get forward to mod3 as input "data_0"
+    
pipe_config[mod1]["output"][1].connect(pipe_config[mod3]["input"]["data_0"])
+
+    # mod2 output(0) would get forward to mod3 as input "data_1"
+    
pipe_config[mod2]["output"][0].connect(pipe_config[mod3]["input"]["data_1"])
+
+    # mod1 output(2) would get forward as final pipeline compute output(1)
+    pipe_config[mod1]["output"][2].connect(pipe_config["output"]["0"])
+
+    # mod3 output(0) would get forward as final pipeline compute output(2)
+    pipe_config[mod3]["output"][0].connect(pipe_config["output"]["1"])
+    """
+    # print configueration (print(pipe_config)), the expect result like 
following.
+    #
+    #Inputs
+    #  |data_0: mod1:data_0
+    #  |data_1: mod2:data_1
+    #
+    #output
+    #  |output(1) : mod1.output(2)
+    #  |output(2) : mod3.output(0)
+    #
+    #connections
+    #  |mod1.output(0)-> mod2.data_0
+    #  |mod1.output(1)-> mod3.data_0
+    #  |mod2.output(0)-> mod3.data_1
+    """
+
+    """
+    # set other parameter.
+    """
+    pipe_config[mod1].set_target(target[0])
+    pipe_config[mod1].set_dev(target[1])
+
+    pipe_config[mod2].set_target("llvm")
+    pipe_config[mod2].set_dev(tvm.cpu(0))
+
+    pipe_config[mod3].set_target("llvm")
+    pipe_config[mod3].set_dev(tvm.cpu(0))
+
+    """
+    # check if the configuration match expectation.
+    """
+    assert pipe_config.get_config() == get_manual_conf([mod1, mod2, mod3], 
target)
+
+    """
+    # Test build and create pipeline module
+    """
+    with relay.build_config(opt_level=3):
+        pipeline_mod_config = pipeline_executor.build(pipe_config)
+
+    pipeline_module = pipeline_executor.create(pipeline_mod_config)
+    assert pipeline_module
+
+
+def pipe_config_check(mod1, mod2, mod3):
+    """
+    # try invalid input/output name exepect runtime error
+    """
+    pipe_error = pipeline_executor.PipelineConfig()
+    with pytest.raises(RuntimeError):
+        pipe_error[mod1]["output"][9]
+
+    with pytest.raises(RuntimeError):
+        pipe_error[mod1]["input"]["data_9"]
+
+    """
+    # try cirle connection , expect runtime error
+    """
+    with pytest.raises(RuntimeError):
+        
pipe_error[mod1]["output"][0].connect(pipe_error[mod2]["input"]["data_0"])
+        
pipe_error[mod2]["output"][0].connect(pipe_error[mod1]["input"]["data_0"])
+
+    """
+    # try wrong module order connection check, expect runtime error.
+    """
+
+    with pytest.raises(RuntimeError):
+        
pipe_error[mod1]["output"][0].connect(pipe_error[mod1]["input"]["data_0"])
+
+    with pytest.raises(RuntimeError):
+        
pipe_error[mod1]["input"]["data_0"].connect(pipe_error[mod1]["input"]["data_0"])
+
+    with pytest.raises(RuntimeError):
+        
pipe_error[mod1]["input"]["data_0"].connect(pipe_error[mod2]["input"]["data_0"])
+
+    with pytest.raises(RuntimeError):
+        pipe_error[mod1]["output"][0].connect(pipe_error["input"]["data_0"])
+
+    with pytest.raises(RuntimeError):
+        pipe_error["input"]["data_0"].connect(pipe_error[mod1]["output"][0])
+
+    with pytest.raises(RuntimeError):
+        pipe_error["output"]["0"].connect(pipe_error[mod1]["output"][0])
+
+
+def test_pipeline():
+    if pipeline_executor.pipeline_executor_enabled():
+        target_list = tvm.testing.enabled_targets()
+        for target in target_list:
+            pipeline(target)
+
+
+if __name__ == "__main__":
+    test_pipeline()

Review comment:
       ```suggestion
   if __name__ == "__main__":
       pytest.main([__file__])
   ```




-- 
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-unsubscr...@tvm.apache.org

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


Reply via email to