You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/10/04 19:30:41 UTC

[GitHub] [tvm] comaniac commented on a change in pull request #9108: [Runtime] Pipeline Executor Second patch, configuration load and executor export/import.

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



##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -86,8 +88,43 @@ class PipelineModule(object):
         Common interface for pipeline executor factory modules.
     """
 
-    def __init__(self, module):
-        self.module = module.module
+    def __init__(self, module=None):
+        self.module = module.module if module else None
+        self._get_num_outputs = None
+        # Get the packed functions from the pipeline executor.
+        self.load_functions()
+
+    def import_from_library(self, config_file_name):
+        """Import files to create pipeline executor.
+
+        Parameters
+        ----------
+        config_file_name : str
+            The configuration file path, the configuration file contains the
+            disk path of the parameter file, library file and JSON file。
+        """
+        # Create a empty PipelineExecutorFactoryModule.
+        pipeline_factory = PipelineExecutorFactoryModule()
+        # Load the configuration file to initialize a PipelineExecutorFactoryModule.
+        pipeline_factory.import_from_library(config_file_name)
+        self.module = pipeline_factory.module
+        # Get packed functions from the pipeline executor.
+        self.load_functions()
+
+    def load_functions(self):
+        # Get functions from the pipeline executor.
+        self._get_num_outputs = self.module["get_num_outputs"] if self.module else None
+
+    def get_num_outputs(self):
+        """Get the number of outputs.
+        Returns
+        -------
+        count : int
+            The number of outputs.
+        """
+        if not self._get_num_outputs:
+            raise RuntimeError(f"The pipeline executor has not been initialized.")
+        return self._get_num_outputs()

Review comment:
       1. `_get_num_outputs`  is not a function?
   2. It seems better to use setter (https://www.python-course.eu/python3_properties.php).

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -501,17 +538,18 @@ class PipelineExecutorFactoryModule(object):
 
     """
 
-    def __init__(self, pipeline_mods, mods_config):
-        mods, config = self.graph_executor_create(pipeline_mods, mods_config)
-        assert (
-            pipeline_executor_enabled()
-        ), "Pipeline executor is not enabled. Please \
-              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
-        pipeline_create = tvm._ffi.get_global_func(
+    def __init__(self, pipeline_libs=None, mods_config=None):
+        self.pipeline_libs = pipeline_libs
+        self.mods_config = mods_config
+        self.pipeline_create = tvm._ffi.get_global_func(

Review comment:
       You set `allow_missing=False`, so the program will crash here without a message.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -86,8 +88,43 @@ class PipelineModule(object):
         Common interface for pipeline executor factory modules.
     """
 
-    def __init__(self, module):
-        self.module = module.module
+    def __init__(self, module=None):
+        self.module = module.module if module else None
+        self._get_num_outputs = None
+        # Get the packed functions from the pipeline executor.
+        self.load_functions()
+
+    def import_from_library(self, config_file_name):
+        """Import files to create pipeline executor.
+
+        Parameters
+        ----------
+        config_file_name : str
+            The configuration file path, the configuration file contains the
+            disk path of the parameter file, library file and JSON file。
+        """
+        # Create a empty PipelineExecutorFactoryModule.

Review comment:
       ```suggestion
           # Create an empty PipelineExecutorFactoryModule.
   ```

##########
File path: src/runtime/pipeline/pipeline_executor.h
##########
@@ -36,25 +43,89 @@ namespace runtime {
  *
  *  This executor can be accessed by various language via TVM runtime PackedFunc API.
  */
-class TVM_DLL PipelineRuntime : public ModuleNode {
+class TVM_DLL PipelineExecutor : public ModuleNode {
  public:
   /*!
    * \Return the type key of the executor.
    */
-  const char* type_key() const final { return "PipelineRuntime"; }
+  const char* type_key() const final { return "PipelineExecutor"; }
   /*!
    * \brief Initialize the pipeline executor with module array and json text.
    * \param modules The module list used for building pipeline.
    * \param pipeline_json The configuration of modules dependencies.
    */
-  void Init(const Array<tvm::runtime::Module>& modules, const std::string& pipeline_json);
+  void Init(const Array<Module>& modules, const std::string& pipeline_json);
   /*!
    * \brief Give frontends an access to packed functions.
    * \param name The name of the function.
    * \param sptr_to_self The pointer to the module node.
    * \return The corresponding packed function.
    */
   virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self);
+
+  /*!
+   * \brief Get the number of outputs.
+   *
+   * \return The number of outputs.
+   */
+  int NumOutputs() const { return num_outputs_; }
+
+ private:
+  /*!\brief The class used to execute pipeline logic*/
+  PipelineFunction pipeline_function_;
+  /*!\brief The Dependency information of each graph runtime module of pipeline.*/
+  PipelineConfig pipeline_config_;
+  /*!\brief The Module information that can get used to create graph runtime.*/
+  ModuleConfig mod_config_;
+  /*!\birief How many outputs are in this pipeline executor.*/
+  size_t num_outputs_ = 0;
+  /*!\brief Json loader.*/
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    while (reader->NextArrayItem()) {
+      std::string key;
+      reader->BeginObject();
+      int mod_idx = 0;
+      std::string lib_name;
+      std::string json_name;
+      std::string params_name;
+      std::string dev;
+      OutputMap output;
+      while (reader->NextObjectItem(&key)) {
+        if (key == "mod_idx") {
+          reader->Read(&mod_idx);
+        }
+        if (key == "lib_name") {
+          reader->Read(&lib_name);
+        }
+
+        if (key == "json_name") {
+          reader->Read(&json_name);
+        }
+
+        if (key == "params_name") {
+          reader->Read(&params_name);
+        }
+
+        if (key == "dev") {
+          reader->Read(&dev);
+        }
+
+        if (key == "output") {
+          reader->Read(&output);
+        }
+      }
+      // Check if mod_idx is read successfully.
+      ICHECK(mod_idx > 0);
+      // Check if the output is read successfully.
+      ICHECK(!output.Empty());

Review comment:
       Add error messages when failed.

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -533,11 +571,81 @@ def graph_executor_create(self, pipeline_mods, mod_config):
             The Modudle configuration.
         """
 
-        mods = []
-        for pipeline_mod in pipeline_mods:
-            mod = graph_executor.GraphModule(
-                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
-            )
-            mods.append(mod.module)
+        # Modules need to be stored in the list named 'mods' in index order.
+        mods = [None for _ in range(len(pipeline_mods))]
+        for lib_index in pipeline_mods:
+            pipeline_lib = pipeline_mods[lib_index]["lib"]
+            dev = pipeline_mods[lib_index]["dev"]
+            lib = graph_executor.GraphModule(pipeline_lib["default"](dev))
+            # Return a module list sorted by lib_index, because lib_index start from 1, use
+            # 'lib_index - 1' here to get the correct index value of module in list.
+            mods[lib_index - 1] = lib.module
 
         return mods, json.dumps(mod_config)
+
+    def export_library(self, directory_path=None):
+        """Export the pipeline executor into disk files.
+
+        Parameters
+        ----------
+        directory_path : str
+            The directory to which these files are exported.
+        """
+        if not self.pipeline_libs:
+            raise RuntimeError(f"The pipeline executor has not been initialized.")
+
+        # If the directory_path is not set, use the temporary path as the file storage
+        # directory_path.
+        if not directory_path:
+            directory_path = tvm.contrib.utils.tempdir().temp_dir

Review comment:
       From its use case, this doesn't look like a temporary directory. Instead, it includes the exported configuration file. In this case, it is improper to have it under `/tmp`.

##########
File path: src/runtime/pipeline/pipeline_executor.cc
##########
@@ -21,31 +21,48 @@
  * \file pipeline_executor.cc
  */
 #include "pipeline_executor.h"
-
 namespace tvm {
 namespace runtime {
-
-void PipelineRuntime::Init(const Array<tvm::runtime::Module>& modules,
-                           const std::string& pipeline_json) {
-  return;
-}
-
-/* GetFunction can not be pure abstract function, implement an empty function for now.
+/*!
+ * \brief Give frontends an access to packed functions.
+ * \param name The name of the function.
+ * \param sptr_to_self The pointer to the module node.
+ * \return The corresponding packed function.
  */
-PackedFunc PipelineRuntime::GetFunction(const std::string& name,
-                                        const ObjectPtr<Object>& sptr_to_self) {
+PackedFunc PipelineExecutor::GetFunction(const std::string& name,
+                                         const ObjectPtr<Object>& sptr_to_self) {
+  if (name == "get_num_outputs") {
+    return PackedFunc(
+        [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->NumOutputs(); });
+  } else {
+    return PackedFunc();
+  }
   return nullptr;
 }
+/*!
+ * \brief Initialize the pipeline executor with module array and json text.

Review comment:
       ```suggestion
    * \brief Initialize the pipeline executor with a list of modules to be pipelined and config in JSON format.
   ```

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -501,17 +538,18 @@ class PipelineExecutorFactoryModule(object):
 
     """
 
-    def __init__(self, pipeline_mods, mods_config):
-        mods, config = self.graph_executor_create(pipeline_mods, mods_config)
-        assert (
-            pipeline_executor_enabled()
-        ), "Pipeline executor is not enabled. Please \
-              re-build TVM with USE_PIPELINE_EXECUTOR=ON"
-        pipeline_create = tvm._ffi.get_global_func(
+    def __init__(self, pipeline_libs=None, mods_config=None):
+        self.pipeline_libs = pipeline_libs
+        self.mods_config = mods_config
+        self.pipeline_create = tvm._ffi.get_global_func(
             "tvm.pipeline_executor.create", allow_missing=False
         )
-        assert pipeline_create
-        self.module = pipeline_create(mods, config)
+        self.module = None
+        # Only create pipeline executor when pipeline_libs, mods_config and
+        # self.pipeline_create are not None.
+        if pipeline_libs and mods_config:
+            graph_executors, config = self.graph_executor_create(pipeline_libs, mods_config)
+            self.module = self.pipeline_create(graph_executors, config)

Review comment:
       Why not just throw out an error otherwise?

##########
File path: src/runtime/pipeline/pipeline_executor.h
##########
@@ -36,25 +43,89 @@ namespace runtime {
  *
  *  This executor can be accessed by various language via TVM runtime PackedFunc API.
  */
-class TVM_DLL PipelineRuntime : public ModuleNode {
+class TVM_DLL PipelineExecutor : public ModuleNode {
  public:
   /*!
    * \Return the type key of the executor.
    */
-  const char* type_key() const final { return "PipelineRuntime"; }
+  const char* type_key() const final { return "PipelineExecutor"; }
   /*!
    * \brief Initialize the pipeline executor with module array and json text.
    * \param modules The module list used for building pipeline.
    * \param pipeline_json The configuration of modules dependencies.
    */
-  void Init(const Array<tvm::runtime::Module>& modules, const std::string& pipeline_json);
+  void Init(const Array<Module>& modules, const std::string& pipeline_json);
   /*!
    * \brief Give frontends an access to packed functions.
    * \param name The name of the function.
    * \param sptr_to_self The pointer to the module node.
    * \return The corresponding packed function.
    */
   virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self);
+
+  /*!
+   * \brief Get the number of outputs.
+   *
+   * \return The number of outputs.
+   */
+  int NumOutputs() const { return num_outputs_; }
+
+ private:
+  /*!\brief The class used to execute pipeline logic*/
+  PipelineFunction pipeline_function_;
+  /*!\brief The Dependency information of each graph runtime module of pipeline.*/
+  PipelineConfig pipeline_config_;
+  /*!\brief The Module information that can get used to create graph runtime.*/
+  ModuleConfig mod_config_;
+  /*!\birief How many outputs are in this pipeline executor.*/
+  size_t num_outputs_ = 0;
+  /*!\brief Json loader.*/
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    while (reader->NextArrayItem()) {
+      std::string key;
+      reader->BeginObject();
+      int mod_idx = 0;
+      std::string lib_name;
+      std::string json_name;
+      std::string params_name;
+      std::string dev;
+      OutputMap output;
+      while (reader->NextObjectItem(&key)) {
+        if (key == "mod_idx") {
+          reader->Read(&mod_idx);
+        }
+        if (key == "lib_name") {
+          reader->Read(&lib_name);
+        }
+
+        if (key == "json_name") {
+          reader->Read(&json_name);
+        }
+
+        if (key == "params_name") {
+          reader->Read(&params_name);
+        }
+
+        if (key == "dev") {
+          reader->Read(&dev);
+        }
+
+        if (key == "output") {
+          reader->Read(&output);
+        }
+      }
+      // Check if mod_idx is read successfully.
+      ICHECK(mod_idx > 0);
+      // Check if the output is read successfully.
+      ICHECK(!output.Empty());
+      pipeline_config_.Insert(mod_idx, output);
+      // Check if there is lib, json and params information.
+      if (!lib_name.empty() && !json_name.empty() && !params_name.empty()) {

Review comment:
       throw an error otherwise?

##########
File path: python/tvm/contrib/pipeline_executor.py
##########
@@ -533,11 +571,81 @@ def graph_executor_create(self, pipeline_mods, mod_config):
             The Modudle configuration.
         """
 
-        mods = []
-        for pipeline_mod in pipeline_mods:
-            mod = graph_executor.GraphModule(
-                pipeline_mod["default"](pipeline_mods[pipeline_mod]["dev"])
-            )
-            mods.append(mod.module)
+        # Modules need to be stored in the list named 'mods' in index order.
+        mods = [None for _ in range(len(pipeline_mods))]
+        for lib_index in pipeline_mods:
+            pipeline_lib = pipeline_mods[lib_index]["lib"]
+            dev = pipeline_mods[lib_index]["dev"]
+            lib = graph_executor.GraphModule(pipeline_lib["default"](dev))
+            # Return a module list sorted by lib_index, because lib_index start from 1, use
+            # 'lib_index - 1' here to get the correct index value of module in list.
+            mods[lib_index - 1] = lib.module
 
         return mods, json.dumps(mod_config)
+
+    def export_library(self, directory_path=None):
+        """Export the pipeline executor into disk files.
+
+        Parameters
+        ----------
+        directory_path : str
+            The directory to which these files are exported.
+        """
+        if not self.pipeline_libs:
+            raise RuntimeError(f"The pipeline executor has not been initialized.")
+
+        # If the directory_path is not set, use the temporary path as the file storage
+        # directory_path.
+        if not directory_path:
+            directory_path = tvm.contrib.utils.tempdir().temp_dir
+
+        # If the directory does not exist, create the directory.
+        if not os.path.exists(directory_path):
+            os.makedirs(directory_path)
+        # Create a configuration copy for export.
+        export_conf = self.mods_config.copy()
+        # Export the library, JSON and parameter into files, then export these files path
+        # into a configuraton file.
+        for lib_index in self.pipeline_libs:
+            mconf = export_conf[lib_index - 1]
+            mconf["lib_name"] = "{}/lib{}.so".format(directory_path, lib_index)
+            mconf["json_name"] = "{}/json{}".format(directory_path, lib_index)
+            mconf["params_name"] = "{}/params{}".format(directory_path, lib_index)
+            mconf["dev"] = "{},{}".format(
+                self.pipeline_libs[lib_index]["dev"].device_type,
+                self.pipeline_libs[lib_index]["dev"].device_id,
+            )
+
+            # Get the graph, lib and parameters from GraphExecutorFactoryModule.
+            graph, lib, params = self.pipeline_libs[lib_index]["lib"]
+            # Export the lib, graph and parameters to disk.
+            lib.export_library(mconf["lib_name"])
+            with open(mconf["json_name"], "w") as file_handle:
+                file_handle.write(graph)
+            with open(mconf["params_name"], "wb") as file_handle:
+                file_handle.write(relay.save_param_dict(params))
+
+        # Export the configuration file to disk.
+        conf_file_name = "{}/config".format(directory_path)
+        with open(conf_file_name, "w") as file_handle:
+            file_handle.write(json.dumps(export_conf))
+
+        return conf_file_name
+
+    def import_from_library(self, config_file_name):

Review comment:
       Better to be `load_library` to follow the graph executor naming convention.

##########
File path: src/runtime/pipeline/pipeline_executor.cc
##########
@@ -21,31 +21,48 @@
  * \file pipeline_executor.cc
  */
 #include "pipeline_executor.h"
-
 namespace tvm {
 namespace runtime {
-
-void PipelineRuntime::Init(const Array<tvm::runtime::Module>& modules,
-                           const std::string& pipeline_json) {
-  return;
-}
-
-/* GetFunction can not be pure abstract function, implement an empty function for now.
+/*!
+ * \brief Give frontends an access to packed functions.
+ * \param name The name of the function.
+ * \param sptr_to_self The pointer to the module node.
+ * \return The corresponding packed function.
  */
-PackedFunc PipelineRuntime::GetFunction(const std::string& name,
-                                        const ObjectPtr<Object>& sptr_to_self) {
+PackedFunc PipelineExecutor::GetFunction(const std::string& name,
+                                         const ObjectPtr<Object>& sptr_to_self) {
+  if (name == "get_num_outputs") {
+    return PackedFunc(
+        [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->NumOutputs(); });
+  } else {
+    return PackedFunc();

Review comment:
       throw an error

##########
File path: src/runtime/pipeline/pipeline_executor.h
##########
@@ -36,25 +43,89 @@ namespace runtime {
  *
  *  This executor can be accessed by various language via TVM runtime PackedFunc API.
  */
-class TVM_DLL PipelineRuntime : public ModuleNode {
+class TVM_DLL PipelineExecutor : public ModuleNode {
  public:
   /*!
    * \Return the type key of the executor.
    */
-  const char* type_key() const final { return "PipelineRuntime"; }
+  const char* type_key() const final { return "PipelineExecutor"; }
   /*!
    * \brief Initialize the pipeline executor with module array and json text.
    * \param modules The module list used for building pipeline.
    * \param pipeline_json The configuration of modules dependencies.
    */
-  void Init(const Array<tvm::runtime::Module>& modules, const std::string& pipeline_json);
+  void Init(const Array<Module>& modules, const std::string& pipeline_json);
   /*!
    * \brief Give frontends an access to packed functions.
    * \param name The name of the function.
    * \param sptr_to_self The pointer to the module node.
    * \return The corresponding packed function.
    */
   virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self);
+
+  /*!
+   * \brief Get the number of outputs.
+   *
+   * \return The number of outputs.
+   */
+  int NumOutputs() const { return num_outputs_; }
+
+ private:
+  /*!\brief The class used to execute pipeline logic*/

Review comment:
       ```suggestion
     /*!\brief The class used to execute pipeline logic.*/
   ```

##########
File path: src/runtime/pipeline/pipeline_executor.h
##########
@@ -36,25 +43,89 @@ namespace runtime {
  *
  *  This executor can be accessed by various language via TVM runtime PackedFunc API.
  */
-class TVM_DLL PipelineRuntime : public ModuleNode {
+class TVM_DLL PipelineExecutor : public ModuleNode {
  public:
   /*!
    * \Return the type key of the executor.
    */
-  const char* type_key() const final { return "PipelineRuntime"; }
+  const char* type_key() const final { return "PipelineExecutor"; }
   /*!
    * \brief Initialize the pipeline executor with module array and json text.
    * \param modules The module list used for building pipeline.
    * \param pipeline_json The configuration of modules dependencies.
    */
-  void Init(const Array<tvm::runtime::Module>& modules, const std::string& pipeline_json);
+  void Init(const Array<Module>& modules, const std::string& pipeline_json);
   /*!
    * \brief Give frontends an access to packed functions.
    * \param name The name of the function.
    * \param sptr_to_self The pointer to the module node.
    * \return The corresponding packed function.
    */
   virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self);
+
+  /*!
+   * \brief Get the number of outputs.
+   *
+   * \return The number of outputs.
+   */
+  int NumOutputs() const { return num_outputs_; }
+
+ private:
+  /*!\brief The class used to execute pipeline logic*/
+  PipelineFunction pipeline_function_;
+  /*!\brief The Dependency information of each graph runtime module of pipeline.*/
+  PipelineConfig pipeline_config_;
+  /*!\brief The Module information that can get used to create graph runtime.*/
+  ModuleConfig mod_config_;
+  /*!\birief How many outputs are in this pipeline executor.*/
+  size_t num_outputs_ = 0;
+  /*!\brief Json loader.*/
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    while (reader->NextArrayItem()) {
+      std::string key;
+      reader->BeginObject();
+      int mod_idx = 0;
+      std::string lib_name;
+      std::string json_name;
+      std::string params_name;
+      std::string dev;
+      OutputMap output;
+      while (reader->NextObjectItem(&key)) {
+        if (key == "mod_idx") {
+          reader->Read(&mod_idx);
+        }
+        if (key == "lib_name") {
+          reader->Read(&lib_name);
+        }
+
+        if (key == "json_name") {
+          reader->Read(&json_name);
+        }
+
+        if (key == "params_name") {
+          reader->Read(&params_name);
+        }
+
+        if (key == "dev") {
+          reader->Read(&dev);
+        }
+
+        if (key == "output") {
+          reader->Read(&output);
+        }

Review comment:
       Can these be the following?
   ```
   if (...) {
   } else if (...) {
   } ...
   ```

##########
File path: src/runtime/pipeline/pipeline_function.h
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#ifndef TVM_RUNTIME_PIPELINE_PIPELINE_FUNCTION_H_
+#define TVM_RUNTIME_PIPELINE_PIPELINE_FUNCTION_H_
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/packed_func.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "pipeline_struct.h"
+namespace tvm {
+namespace runtime {
+/*!
+ * \brief The class that executes the pipeline logic,it is used to initialize the thread pool,
+    execute and schedule pipeline tasks, allocate and manage memory, etc.
+ */
+class PipelineFunction {

Review comment:
       The name `PipelineFunction` looks a bit weird to me.  Why not just have `PipelineExecutor`?

##########
File path: src/runtime/pipeline/pipeline_function.cc
##########
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include "pipeline_function.h"
+
+#include <utility>
+#include <vector>
+namespace tvm {
+namespace runtime {
+/*!
+ * \brief Initialize the pipeline.
+ * \param modules The list of graph executor module.
+ * \param pipeline_conf The Dependency information of each graph executor module.
+ * \param mod_config The config information that generated by the export library function call.
+ */
+size_t PipelineFunction::PipelineInit(Array<Module> modules, const PipelineConfig& pipeline_config,
+                                      const ModuleConfig& mod_config) {

Review comment:
       Didn't understand the semantic of this function. After reviewing the rest parts, I realized that it's mostly due to the naming. I agree with @masahi that `graph_executors_` should not be named to `executor`. Maybe just call it `modules`, or `ordered_modules`.




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

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

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