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/09/21 07:04:14 UTC

[GitHub] [tvm] junrushao1994 opened a new pull request #9061: [Meta Schedule][M3c] Database

junrushao1994 opened a new pull request #9061:
URL: https://github.com/apache/tvm/pull/9061


   This PR is part of the meta schedule project (#8473) that adds a workload registry, as well as
   a generic database of tuning records. This feature is future-compatible with dynamic shape
   auto-tuning.
   
   Depend on PR #9059.
   
   
   Co-authored-by: Xiyou Zhou <xi...@octoml.ai>
   Co-authored-by: Bohan Hou <32...@users.noreply.github.com>
   Co-authored-by: Ruihang Lai <la...@qq.com>
   Co-authored-by: Hongyi Jin <32...@qq.com>
   Co-authored-by: Wuwei Lin <wu...@apache.org>
   Co-authored-by: Siyuan Feng <Hz...@sjtu.edu.cn>
   


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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: python/tvm/meta_schedule/database/json_file.py
##########
@@ -0,0 +1,37 @@
+# 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.
+"""The default database that uses a JSON File to store tuning records"""
+from tvm._ffi import register_object
+
+from .. import _ffi_api
+from .database import Database
+
+
+@register_object("meta_schedule.JSONFile")
+class JSONFile(Database):
+    def __init__(
+        self,
+        record_path: str,
+        workload_path: str,
+        allow_missing: bool = True,
+    ) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.DatabaseJSONFile,  # type: ignore # pylint: disable=no-member
+            record_path,
+            workload_path,
+            allow_missing,
+        )

Review comment:
       I'm not sure I understand the question. This design follows the convention of AutoTVM and AutoScheduler where each line of a JSON file is a tuning record. Given the design is unlikeable in production, we allow developers to inherit from `PyDatabase` to interface with their own database, and this isn't limited to using JSON - they can actually use any format they like to serialize these records, as long as they override the 4 APIs required in `PyDatabase`




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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3c] Database

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


   > It seems the direct use of the Database is all wrapped in a JSON-related file object, which kind of confuses me, would you mind providing a brief explanation of the class structure somewhere?
   
   Hey thanks @mbrookhart for asking! Let's focus on the database API and ignore the workload registry part which will be removed according to previous discussion with Cody in this thread.
   
   * The base abstract class is `Database`, which has 4 pure virtual methods: `commit_workload`, `commit_tuning_record`, `get_top_k`, `size`
   * The system interacts with any subclass of `Database` by calling these 4 methods
   * To implement a concrete subclass of `Database`: in pure C++, we can just inherit Database and do so; in pure python, we provide `PyDatabase` which plays some tricks so that everything can be implemented in pure python
   * `JSONFile` is a simple default implementation of `Database` for open source


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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3b] Database

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


   Thanks everybody for the review :-)


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

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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #9061: [Meta Schedule][M3c] Database

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


   > It seems the direct use of the Database is all wrapped in a JSON-related file object, which kind of confuses me, would you mind providing a brief explanation of the class structure somewhere?
   
   Hey thanks @mbrookhart for asking! Let's focus on the database API and ignore the workload registry part (which will be removed according to previous discussion with Cody in this thread).
   
   * The base abstract class is `Database`, which has 4 pure virtual methods: `commit_workload`, `commit_tuning_record`, `get_top_k`, `size`
   * The system interacts with any subclass of `Database` by calling these 4 methods
   * To implement a concrete subclass of `Database`: in pure C++, we can just inherit Database and do so; in pure python, we provide `PyDatabase` to inherit from, where we plays some tricks, so that everything can be implemented in pure python
   * `JSONFile` is a simple default implementation of `Database` for open source


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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3c] Database

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


   @mbrookhart @comaniac I updated the patch according to your comments. Major changes:
   - Remove `WorkloadRegistry`, `WorkloadToken` and the annoying `token_id_`
   - Rename `lookup_or_add` => `commit_workload`
   - Rename `add` => `commit_tuning_record`
   - Various refactors to consolidate more stuff
   
   Let me know what you guys think :-)


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

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

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



[GitHub] [tvm] mbrookhart commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited

Review comment:
       :+1: I guess we can provide access on the python frontend anyway. Maybe just make a comment to that effect?




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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3c] Database

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


   BTW, here is the convention we are using in TensorIR and meta schedule project:
   - An abstract base class that describe how the interface looks like, e.g. `Schedule`, `Database`, `Builder`, `Runner`
   - Concrete implementation as subclasses are hidden in `src/` folder, and mostly hidden in cc files, which are not directly visible to developers. For example, `PyDatabase`, `ConcreteSchedule`
   - The abstract class provides factory methods to create these concrete implementations, whose name is consistent with the names of the subclasses. For example:
   
   ```C++
   class Database : ... {
    public:
     static Database JSONFile(...); // TBD: Rename 
     static Database SQLiteDB(...);
   };
   ```
   
   And then another problem here is what name should we change `JSONFile` for


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

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

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



[GitHub] [tvm] comaniac commented on pull request #9061: [Meta Schedule][M3c] Database

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


   Will try to find time reviewing this PR this afternoon or tomorrow.


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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3c] Database

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


   Per discussion with @comaniac.
   
   The naming change:
   - `lookup_or_add` seems pretty random, and we should rename it as `commit_workload` which is more consistent with database terminologies
   - `add` seems okay but `commit_tuning_record` sounds better and more consistent with `commit_workload`
   
   Minor change in the code structure:
   - Consolidate `WorkloadRegistry` into `JSONFile` database, so that there is only one unified database API, and this could mimic the two-table database behavior
   


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

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

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



[GitHub] [tvm] comaniac commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: python/tvm/meta_schedule/database/json_file.py
##########
@@ -0,0 +1,61 @@
+# 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.
+"""The default database that uses a JSON File to store tuning records"""
+from tvm._ffi import register_object
+
+from .. import _ffi_api
+from .database import Database
+
+
+@register_object("meta_schedule.JSONFile")
+class JSONFile(Database):
+    """The class of tuning records.

Review comment:
       According to the discussion, this is more like a database implementation, so it might be better to say something like "The JSON-file-based database".

##########
File path: python/tvm/meta_schedule/database/database.py
##########
@@ -0,0 +1,240 @@
+# 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.
+"""Tuning record database"""
+from typing import Any, List
+
+from tvm._ffi import register_object
+from tvm.ir.module import IRModule
+from tvm.runtime import Object
+from tvm.target import Target
+from tvm.tir.schedule import Trace
+
+from .. import _ffi_api
+from ..arg_info import ArgInfo
+from ..utils import _json_de_tvm
+
+
+@register_object("meta_schedule.Workload")
+class Workload(Object):
+    """A workload, i.e. an IRModule and its structural hash.
+
+    Parameters
+    ----------
+    mod : IRModule
+        The workload's IRModule
+    """
+
+    mod: IRModule
+
+    def __init__(self, mod: IRModule) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.Workload,  # type: ignore # pylint: disable=no-member
+            mod,
+        )
+
+    def as_json(self) -> Any:
+        """Export the workload to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _json_de_tvm(_ffi_api.WorkloadAsJSON(self))  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any) -> "Workload":
+        """Create a workload from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.WorkloadFromJSON(json_obj)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.TuningRecord")
+class TuningRecord(Object):
+    """The class of tuning records.
+
+    Parameters
+    ----------
+    trace : tvm.ir.Trace
+        The trace of the tuning record.
+    run_secs : List[float]
+        The run time of the tuning record.
+    workload : Workload
+        The workload of the tuning record.
+    target : Target
+        The target of the tuning record.
+    args_info : List[ArgInfo]
+        The argument information of the tuning record.
+    """
+
+    trace: Trace
+    run_secs: List[float]
+    workload: Workload
+    target: Target
+    args_info: List[ArgInfo]
+
+    def __init__(
+        self,
+        trace: Trace,
+        run_secs: List[float],
+        workload: Workload,
+        target: Target,
+        args_info: List[ArgInfo],
+    ) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningRecord,  # type: ignore # pylint: disable=no-member
+            trace,
+            run_secs,
+            workload,
+            target,
+            args_info,
+        )
+
+    def as_json(self) -> Any:
+        """Export the tuning record to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _json_de_tvm(_ffi_api.TuningRecordAsJSON(self))  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any, workload: Workload) -> "TuningRecord":
+        """Create a tuning record from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+        workload : Workload
+            The workload.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.TuningRecordFromJSON(json_obj, workload)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.Database")
+class Database(Object):
+    """The abstract database interface."""
+
+    def commit_workload(self, mod: IRModule) -> Workload:
+        """Look up or add workload to the database if missing.
+
+        Parameters
+        ----------
+        mod : IRModule
+            The IRModule to be searched for or added.
+
+        Returns
+        -------
+        workload : Workload
+            The workload corresponding to the given IRModule.
+        """
+        return _ffi_api.DatabaseCommitWorkload(self, mod)  # type: ignore # pylint: disable=no-member
+
+    def commit_tuning_record(self, record: TuningRecord) -> None:
+        """Add a tuning record to the database.

Review comment:
       ```suggestion
           """Commit a tuning record to the database.
   ```
   Also it might be better to mention what will happen if the same tuning record is committed twice.

##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,275 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief A workload, i.e. an IRModule and its structural hash. */
+class WorkloadNode : public runtime::Object {
+ public:
+  /*! \brief The type of structural hash */
+  using THashCode = size_t;
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  THashCode shash;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    // `shash` is not visited because TVM FFI doesn't support uint64_t
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.Workload";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadNode, runtime::Object);
+
+  /*!
+   * \brief Export the workload to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief Managed reference to WorkloadNode.
+ *  \sa WorkloadNode
+ */
+class Workload : public runtime::ObjectRef {
+ public:
+  using THashCode = WorkloadNode::THashCode;
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   */
+  TVM_DLL explicit Workload(IRModule mod);
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   */
+  TVM_DLL explicit Workload(IRModule mod, THashCode shash);
+  /*!
+   * \brief Create a workload from a json object.
+   * \param json_obj The json object.
+   * \return The created workload.
+   */
+  TVM_DLL static Workload FromJSON(const ObjectRef& json_obj);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(Workload, runtime::ObjectRef, WorkloadNode);
+};
+
+/*! \brief The hash method for Workload */
+struct WorkloadHash {
+  size_t operator()(const Workload& a) const { return a->shash; }
+};
+
+/*! \brief The equality check for Workload */
+struct WorkloadEqual {
+  bool operator()(const Workload& a, const Workload& b) const {
+    return a->shash == b->shash && tvm::StructuralEqual()(a->mod, b->mod);
+  }
+};
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload. */
+  Workload workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, serialized target, and
+   * argument information.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, Workload workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param workload The workload.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const Workload& workload);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  virtual Workload CommitWorkload(const IRModule& mod) = 0;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void CommitTuningRecord(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const Workload& workload, int top_k) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `CommitWorkload` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  using FCommitWorkload = runtime::TypedPackedFunc<Workload(const IRModule&)>;
+  /*!
+   * \brief The function type of `CommitTuningRecord` method.
+   * \param record The tuning record to be added.
+   */
+  using FCommitTuningRecord = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const Workload&, int)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `CommitWorkload` function. */
+  FCommitWorkload f_commit_workload;
+  /*! \brief The packed function to the `CommitTuningRecord` function. */
+  FCommitTuningRecord f_commit_tuning_record;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_commit_workload` is not visited
+    // `f_commit_tuning_record` is not visited
+    // `f_get_top_k` is not visited
+    // `f_size` is not visited
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.PyDatabase";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PyDatabaseNode, DatabaseNode);
+
+  Workload CommitWorkload(const IRModule& mod) final { return f_commit_workload(mod); }
+
+  void CommitTuningRecord(const TuningRecord& record) final { f_commit_tuning_record(record); }
+
+  Array<TuningRecord> GetTopK(const Workload& workload, int top_k) final {
+    return f_get_top_k(workload, top_k);
+  }
+
+  int64_t Size() final { return f_size(); }
+};
+
+/*!
+ * \brief Managed reference to DatabaseNode.
+ * \sa DatabaseNode
+ */
+class Database : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Create a default database that uses JSON file for tuning records.
+   * \param path_workload The path to the workload table.
+   * \param path_tuning_record The path to the database table.
+   * \param allow_missing Whether to create new file when the given path is not found.
+   */
+  TVM_DLL static Database JSONFile(String path_workload, String path_tuning_record,
+                                   bool allow_missing);

Review comment:
       After the refactoring the logic is clearer to me, and I also realized that this API is a bit confusing.
   Per offline discussion with @junrushao1994, this is more like one of the factory methods. In the future, we might have:
   
   ```
   class Database {
     static Database JSONFile(...);
     static Database SQLite(...);
   };
   ```
   
   And uses might be able to initialize a DB for their needs: `Database::SQLite(...)`.
   
   Accordingly, it might be better to consistently name them as database names. For example, we could name `JSONFile` to be `JsonDB`, but I'm bad at naming so would be happy to have better ideas from others. cc @mbrookhart 

##########
File path: python/tvm/meta_schedule/database/database.py
##########
@@ -0,0 +1,240 @@
+# 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.
+"""Tuning record database"""
+from typing import Any, List
+
+from tvm._ffi import register_object
+from tvm.ir.module import IRModule
+from tvm.runtime import Object
+from tvm.target import Target
+from tvm.tir.schedule import Trace
+
+from .. import _ffi_api
+from ..arg_info import ArgInfo
+from ..utils import _json_de_tvm
+
+
+@register_object("meta_schedule.Workload")
+class Workload(Object):
+    """A workload, i.e. an IRModule and its structural hash.
+
+    Parameters
+    ----------
+    mod : IRModule
+        The workload's IRModule
+    """
+
+    mod: IRModule
+
+    def __init__(self, mod: IRModule) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.Workload,  # type: ignore # pylint: disable=no-member
+            mod,
+        )
+
+    def as_json(self) -> Any:
+        """Export the workload to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _json_de_tvm(_ffi_api.WorkloadAsJSON(self))  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any) -> "Workload":
+        """Create a workload from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.WorkloadFromJSON(json_obj)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.TuningRecord")
+class TuningRecord(Object):
+    """The class of tuning records.
+
+    Parameters
+    ----------
+    trace : tvm.ir.Trace
+        The trace of the tuning record.
+    run_secs : List[float]
+        The run time of the tuning record.
+    workload : Workload
+        The workload of the tuning record.
+    target : Target
+        The target of the tuning record.
+    args_info : List[ArgInfo]
+        The argument information of the tuning record.
+    """
+
+    trace: Trace
+    run_secs: List[float]
+    workload: Workload
+    target: Target
+    args_info: List[ArgInfo]
+
+    def __init__(
+        self,
+        trace: Trace,
+        run_secs: List[float],
+        workload: Workload,
+        target: Target,
+        args_info: List[ArgInfo],
+    ) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningRecord,  # type: ignore # pylint: disable=no-member
+            trace,
+            run_secs,
+            workload,
+            target,
+            args_info,
+        )
+
+    def as_json(self) -> Any:
+        """Export the tuning record to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _json_de_tvm(_ffi_api.TuningRecordAsJSON(self))  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any, workload: Workload) -> "TuningRecord":
+        """Create a tuning record from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+        workload : Workload
+            The workload.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.TuningRecordFromJSON(json_obj, workload)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.Database")
+class Database(Object):
+    """The abstract database interface."""
+
+    def commit_workload(self, mod: IRModule) -> Workload:
+        """Look up or add workload to the database if missing.

Review comment:
       ```suggestion
           """Commit a workload to the database if missing.
   ```

##########
File path: tests/python/unittest/test_meta_schedule_database.py
##########
@@ -0,0 +1,274 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=missing-module-docstring,missing-function-docstring,missing-class-docstring
+"""Test Meta Schedule Database"""
+import os.path as osp
+import sys
+import tempfile
+from typing import Callable
+
+import pytest
+
+import tvm
+from tvm import tir
+from tvm.ir.module import IRModule
+from tvm.meta_schedule.arg_info import ArgInfo
+from tvm.meta_schedule.database import JSONFile, TuningRecord
+from tvm.script import ty
+from tvm.tir import Schedule
+
+# pylint: disable=invalid-name,no-member,line-too-long,too-many-nested-blocks,no-self-argument
+# fmt: off
+
+@tvm.script.tir
+class Matmul:
+    def main(a: ty.handle, b: ty.handle, c: ty.handle) -> None:
+        tir.func_attr({"global_symbol": "main"})
+        A = tir.match_buffer(a, (1024, 1024), "float32")
+        B = tir.match_buffer(b, (1024, 1024), "float32")
+        C = tir.match_buffer(c, (1024, 1024), "float32")
+        with tir.block([1024, 1024, tir.reduce_axis(0, 1024)], "matmul") as [vi, vj, vk]:
+            with tir.init():
+                C[vi, vj] = 0.0
+            C[vi, vj] = C[vi, vj] + A[vi, vk] * B[vk, vj]
+
+
+@tvm.script.tir
+class MatmulRelu:
+    def main(a: ty.handle, b: ty.handle, d: ty.handle) -> None:  # pylint: disable=no-self-argument
+        tir.func_attr({"global_symbol": "main", "tir.noalias": True})
+        A = tir.match_buffer(a, (16, 16), "float32")
+        B = tir.match_buffer(b, (16, 16), "float32")
+        D = tir.match_buffer(d, (16, 16), "float32")
+        C = tir.alloc_buffer((16, 16), "float32")
+        with tir.block([16, 16, tir.reduce_axis(0, 16)], "matmul") as [vi, vj, vk]:
+            with tir.init():
+                C[vi, vj] = 0.0
+            C[vi, vj] = C[vi, vj] + A[vi, vk] * B[vk, vj]
+        with tir.block([16, 16], "relu") as [vi, vj]:
+            D[vi, vj] = tir.max(C[vi, vj], 0.0)
+
+
+# fmt: on
+# pylint: enable=invalid-name,no-member,line-too-long,too-many-nested-blocks,no-self-argument
+
+
+def _schedule_matmul(sch: Schedule):
+    block = sch.get_block("matmul")
+    i, j, k = sch.get_loops(block=block)
+    i_tiles = [1, 1, 2, 512]
+    j_tiles = [1, 512, 1, 2]
+    k_tiles = [256, 4]
+    i_0, i_1, i_2, i_3 = sch.split(loop=i, factors=i_tiles)
+    j_0, j_1, j_2, j_3 = sch.split(loop=j, factors=j_tiles)
+    k_0, k_1 = sch.split(loop=k, factors=k_tiles)
+    sch.reorder(i_0, j_0, i_1, j_1, k_0, i_2, j_2, k_1, i_3, j_3)
+
+
+def _create_schedule(mod: IRModule, sch_fn: Callable[[Schedule], None]) -> Schedule:
+    sch = tir.Schedule(mod=mod, debug_mask="all")
+    sch_fn(sch)
+    return sch
+
+
+def _create_tmp_database(tmpdir: str) -> JSONFile:
+    path_workload = osp.join(tmpdir, "workloads.json")
+    path_tuning_record = osp.join(tmpdir, "tuning_records.json")
+    return JSONFile(path_workload, path_tuning_record)

Review comment:
       Echo again. It seems much more straightforward to write `return JsonDB(path_workload, path_tuning_record)` here.




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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,275 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief A workload, i.e. an IRModule and its structural hash. */
+class WorkloadNode : public runtime::Object {
+ public:
+  /*! \brief The type of structural hash */
+  using THashCode = size_t;
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  THashCode shash;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    // `shash` is not visited because TVM FFI doesn't support uint64_t
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.Workload";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadNode, runtime::Object);
+
+  /*!
+   * \brief Export the workload to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief Managed reference to WorkloadNode.
+ *  \sa WorkloadNode
+ */
+class Workload : public runtime::ObjectRef {
+ public:
+  using THashCode = WorkloadNode::THashCode;
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   */
+  TVM_DLL explicit Workload(IRModule mod);
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   */
+  TVM_DLL explicit Workload(IRModule mod, THashCode shash);
+  /*!
+   * \brief Create a workload from a json object.
+   * \param json_obj The json object.
+   * \return The created workload.
+   */
+  TVM_DLL static Workload FromJSON(const ObjectRef& json_obj);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(Workload, runtime::ObjectRef, WorkloadNode);
+};
+
+/*! \brief The hash method for Workload */
+struct WorkloadHash {
+  size_t operator()(const Workload& a) const { return a->shash; }
+};
+
+/*! \brief The equality check for Workload */
+struct WorkloadEqual {
+  bool operator()(const Workload& a, const Workload& b) const {
+    return a->shash == b->shash && tvm::StructuralEqual()(a->mod, b->mod);
+  }
+};
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload. */
+  Workload workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, serialized target, and
+   * argument information.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, Workload workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param workload The workload.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const Workload& workload);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  virtual Workload CommitWorkload(const IRModule& mod) = 0;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void CommitTuningRecord(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const Workload& workload, int top_k) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `CommitWorkload` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  using FCommitWorkload = runtime::TypedPackedFunc<Workload(const IRModule&)>;
+  /*!
+   * \brief The function type of `CommitTuningRecord` method.
+   * \param record The tuning record to be added.
+   */
+  using FCommitTuningRecord = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const Workload&, int)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `CommitWorkload` function. */
+  FCommitWorkload f_commit_workload;
+  /*! \brief The packed function to the `CommitTuningRecord` function. */
+  FCommitTuningRecord f_commit_tuning_record;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_commit_workload` is not visited
+    // `f_commit_tuning_record` is not visited
+    // `f_get_top_k` is not visited
+    // `f_size` is not visited
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.PyDatabase";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PyDatabaseNode, DatabaseNode);
+
+  Workload CommitWorkload(const IRModule& mod) final { return f_commit_workload(mod); }
+
+  void CommitTuningRecord(const TuningRecord& record) final { f_commit_tuning_record(record); }
+
+  Array<TuningRecord> GetTopK(const Workload& workload, int top_k) final {
+    return f_get_top_k(workload, top_k);
+  }
+
+  int64_t Size() final { return f_size(); }
+};
+
+/*!
+ * \brief Managed reference to DatabaseNode.
+ * \sa DatabaseNode
+ */
+class Database : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Create a default database that uses JSON file for tuning records.
+   * \param path_workload The path to the workload table.
+   * \param path_tuning_record The path to the database table.
+   * \param allow_missing Whether to create new file when the given path is not found.
+   */
+  TVM_DLL static Database JSONFile(String path_workload, String path_tuning_record,
+                                   bool allow_missing);

Review comment:
       I was thinking things like `DefaultDBBasedOnTwoJSONFilesAsTables` but it is obvious not a good name :-(




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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: src/meta_schedule/database/json_file.cc
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 <set>
+
+#include "../utils.h"
+#include "tvm/node/structural_equal.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The struct defining comparison function of sorting by mean run seconds. */
+struct SortByMeanRunSecs {

Review comment:
       What about `SortTuningRecordByMeanRunSecs`? I'm really bad at naming lol




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

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

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



[GitHub] [tvm] junrushao1994 merged pull request #9061: [Meta Schedule][M3b] Database

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


   


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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3b] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,275 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief A workload, i.e. an IRModule and its structural hash. */
+class WorkloadNode : public runtime::Object {
+ public:
+  /*! \brief The type of structural hash */
+  using THashCode = size_t;
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  THashCode shash;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    // `shash` is not visited because TVM FFI doesn't support uint64_t
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.Workload";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadNode, runtime::Object);
+
+  /*!
+   * \brief Export the workload to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief Managed reference to WorkloadNode.
+ *  \sa WorkloadNode
+ */
+class Workload : public runtime::ObjectRef {
+ public:
+  using THashCode = WorkloadNode::THashCode;
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   */
+  TVM_DLL explicit Workload(IRModule mod);
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   */
+  TVM_DLL explicit Workload(IRModule mod, THashCode shash);
+  /*!
+   * \brief Create a workload from a json object.
+   * \param json_obj The json object.
+   * \return The created workload.
+   */
+  TVM_DLL static Workload FromJSON(const ObjectRef& json_obj);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(Workload, runtime::ObjectRef, WorkloadNode);
+};
+
+/*! \brief The hash method for Workload */
+struct WorkloadHash {
+  size_t operator()(const Workload& a) const { return a->shash; }
+};
+
+/*! \brief The equality check for Workload */
+struct WorkloadEqual {
+  bool operator()(const Workload& a, const Workload& b) const {
+    return a->shash == b->shash && tvm::StructuralEqual()(a->mod, b->mod);
+  }
+};
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload. */
+  Workload workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, serialized target, and
+   * argument information.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, Workload workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param workload The workload.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const Workload& workload);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  virtual Workload CommitWorkload(const IRModule& mod) = 0;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void CommitTuningRecord(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const Workload& workload, int top_k) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `CommitWorkload` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  using FCommitWorkload = runtime::TypedPackedFunc<Workload(const IRModule&)>;
+  /*!
+   * \brief The function type of `CommitTuningRecord` method.
+   * \param record The tuning record to be added.
+   */
+  using FCommitTuningRecord = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const Workload&, int)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `CommitWorkload` function. */
+  FCommitWorkload f_commit_workload;
+  /*! \brief The packed function to the `CommitTuningRecord` function. */
+  FCommitTuningRecord f_commit_tuning_record;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_commit_workload` is not visited

Review comment:
       Thanks! Yep just added the comment




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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;

Review comment:
       Per discussion with Cody, I am going to remove the workload registry thing, which can be confusing :-)




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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3b] Database

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


   Hey thanks guys for the discussion! Let's go with the name `JSONDatabase` then 😄 


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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited
+  }
+
+  void Add(const TuningRecord& record) final { f_add(record); }
+
+  Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) final {
+    return f_get_top_k(workload, top_k);
+  }
+
+  WorkloadToken LookupOrAdd(const IRModule& mod) final { return f_lookup_or_add(mod); }
+
+  int64_t Size() final { return f_size(); }
+
+  static constexpr const char* _type_key = "meta_schedule.PyDatabase";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PyDatabaseNode, DatabaseNode);
+};
+
+/*!
+ * \brief Managed reference to DatabaseNode.
+ * \sa DatabaseNode
+ */
+class Database : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Create a default database that uses JSON file for tuning records.
+   * \param record_path The path to the database file.
+   * \param workload_path The path to the workload registry file.
+   * \param allow_missing Whether to create new file when the given path is not found.
+   */
+  TVM_DLL static Database JSONFile(String record_path, String workload_path, bool allow_missing);

Review comment:
       Yeah maybe useful for OSS when people want to tune from scratch




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

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

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



[GitHub] [tvm] zxybazh commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited

Review comment:
       These are packed functions, I think they are not accessible through reflection.




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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3c] Database

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


   @mbrookhart Yeah indeed `JSONFile` can be very confusing given JSON means very particular thing. Would love to hear about your proposals


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

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

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



[GitHub] [tvm] comaniac commented on pull request #9061: [Meta Schedule][M3c] Database

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


   Took a fast pass and had an offline discussion with @junrushao1994.  Two major points:
   1. The APIs are bit confusing (e.g.,`lookup_or_add` and `add`). It might be better to use something like `commit_workload` and `commit_record`. Meanwhile, since we may add more database implementations in the future (e.g., sqlite), this general API can also make sure we won't need to refine it again in AutoTIR.
   2. Better to remove workload registry accordingly to make the design simple.


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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited

Review comment:
       yeah because tvm doesn't support reflection with functors




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

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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #9061: [Meta Schedule][M3c] Database

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


   > It seems the direct use of the Database is all wrapped in a JSON-related file object, which kind of confuses me, would you mind providing a brief explanation of the class structure somewhere?
   
   Hey thanks @mbrookhart for asking! Let's focus on the database API and ignore the workload registry part (which will be removed according to previous discussion with Cody in this thread).
   
   * The base abstract class is `Database`, which has 4 pure virtual methods: `commit_workload`, `commit_tuning_record`, `get_top_k`, `size`
   * The system interacts with any subclass of `Database` by calling these 4 methods
   * To implement a concrete subclass of `Database`: in pure C++, we can just inherit Database and do so; in pure python, we provide `PyDatabase` to inherit from, where we plays some tricks, so that everything can be implemented in pure python
   * `JSONFile` is a simple default implementation of `Database` for open source
   * Developers can provide other implementations too by inheriting `Database` or `PyDatabase`, for example, database backed by SQLite, etc, as long as the class implements the 4 methods


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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3b] Database

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






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

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

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



[GitHub] [tvm] mbrookhart commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited

Review comment:
       Can you put a comment as to why they are not visited?

##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    v->Visit("shash", &shash);
+    // `token_id_` is not visited

Review comment:
       Why isn't it visited?

##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    v->Visit("shash", &shash);
+    // `token_id_` is not visited
+  }
+
+  /*!
+   * \brief Export the workload token to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.WorkloadToken";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadTokenNode, runtime::Object);
+};
+
+/*!
+ * \brief Managed reference to WorkloadTokenNode.
+ *  \sa WorkloadTokenNode
+ */
+class WorkloadToken : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Constructor of WorkloadToken.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   * \param token_id The workload's token id.
+   */
+  TVM_DLL explicit WorkloadToken(IRModule mod, String shash, int64_t token_id);
+
+  /*!
+   * \brief Create a workload token from a json object.
+   * \param json_obj The json object.
+   * \param token_id The workload's token id.
+   * \return The created workload token.
+   */
+  TVM_DLL static WorkloadToken FromJSON(const ObjectRef& json_obj, int64_t token_id);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(WorkloadToken, runtime::ObjectRef, WorkloadTokenNode);
+};
+
+/*! \brief The class for workload registry. */
+class WorkloadRegistryNode : public runtime::Object {
+ public:
+  /*! \brief The workload registry's storage path. */
+  String path;
+  /*! \brief The map from workload to its corresponding workload token id. */
+  std::unordered_map<IRModule, int64_t, tvm::StructuralHash, tvm::StructuralEqual> mod2token_id_;
+  /*! \brief The list of workload tokens. */
+  std::vector<WorkloadToken> workload_tokens_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("path", &path);
+    // `mod2token_id_` is not visited
+    // `workload_tokens_` is not visited
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.WorkloadRegistry";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadRegistryNode, runtime::Object);
+
+ public:

Review comment:
       Don't need a second public.

##########
File path: python/tvm/meta_schedule/workload_registry.py
##########
@@ -0,0 +1,139 @@
+# 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.
+"""Workload Registry"""
+from typing import Any, Tuple
+
+from tvm._ffi import register_object
+from tvm.ir import IRModule
+from tvm.runtime import Object
+
+from . import _ffi_api
+
+
+@register_object("meta_schedule.WorkloadToken")
+class WorkloadToken(Object):
+    """The class of workload tokens.
+
+    Parameters
+    ----------
+    mod : tvm.ir.IRModule
+        The IRModule of the workload.
+    shash : str
+        The structural hash of the workload.
+    """
+
+    mod: IRModule
+    shash: str
+
+    def __init__(self, mod: IRModule, shash: str, token_id: int) -> None:

Review comment:
       token_id isn't documented?

##########
File path: src/meta_schedule/database/json_file.cc
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 <set>
+
+#include "../utils.h"
+#include "tvm/node/structural_equal.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The struct defining comparison function of sorting by mean run seconds. */
+struct SortByMeanRunSecs {
+  static double Mean(const Array<FloatImm>& a) {
+    ICHECK(!a.empty());
+    double sum = 0.0;
+    for (const FloatImm& i : a) {
+      sum += i->value;
+    }
+    return sum / a.size();
+  }
+
+  bool operator()(const TuningRecord& a, const TuningRecord& b) const {
+    double a_time = Mean(a->run_secs);
+    double b_time = Mean(b->run_secs);
+    return a_time < b_time;
+  }
+};
+
+/*! \brief The default database implementation. */
+class JSONFileNode : public DatabaseNode {
+ public:
+  /*! \brief The path to store or load database records. */
+  String record_path;
+  /*! \brief The workload registry. */
+  WorkloadRegistry reg{nullptr};
+  /*! \brief The database records organized using a set. */
+  std::multiset<TuningRecord, SortByMeanRunSecs> records_;

Review comment:
       I was not familiar with this stl data structure, this is a really cool use!

##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    v->Visit("shash", &shash);
+    // `token_id_` is not visited
+  }
+
+  /*!
+   * \brief Export the workload token to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.WorkloadToken";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadTokenNode, runtime::Object);
+};
+
+/*!
+ * \brief Managed reference to WorkloadTokenNode.
+ *  \sa WorkloadTokenNode
+ */
+class WorkloadToken : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Constructor of WorkloadToken.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   * \param token_id The workload's token id.
+   */
+  TVM_DLL explicit WorkloadToken(IRModule mod, String shash, int64_t token_id);
+
+  /*!
+   * \brief Create a workload token from a json object.
+   * \param json_obj The json object.
+   * \param token_id The workload's token id.
+   * \return The created workload token.
+   */
+  TVM_DLL static WorkloadToken FromJSON(const ObjectRef& json_obj, int64_t token_id);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(WorkloadToken, runtime::ObjectRef, WorkloadTokenNode);
+};
+
+/*! \brief The class for workload registry. */
+class WorkloadRegistryNode : public runtime::Object {

Review comment:
       It doesn't look like this registry is a singleton? Is Registry the right name?

##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    v->Visit("shash", &shash);
+    // `token_id_` is not visited
+  }
+
+  /*!
+   * \brief Export the workload token to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.WorkloadToken";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadTokenNode, runtime::Object);
+};
+
+/*!
+ * \brief Managed reference to WorkloadTokenNode.
+ *  \sa WorkloadTokenNode
+ */
+class WorkloadToken : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Constructor of WorkloadToken.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   * \param token_id The workload's token id.

Review comment:
       Again, what's a token id

##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;

Review comment:
       Can you give more documentation on what the token id is? I can't tell it's use from this comment. I understand IR module and hash, but not token id.

##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited
+  }
+
+  void Add(const TuningRecord& record) final { f_add(record); }
+
+  Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) final {
+    return f_get_top_k(workload, top_k);
+  }
+
+  WorkloadToken LookupOrAdd(const IRModule& mod) final { return f_lookup_or_add(mod); }
+
+  int64_t Size() final { return f_size(); }
+
+  static constexpr const char* _type_key = "meta_schedule.PyDatabase";
+  TVM_DECLARE_FINAL_OBJECT_INFO(PyDatabaseNode, DatabaseNode);
+};
+
+/*!
+ * \brief Managed reference to DatabaseNode.
+ * \sa DatabaseNode
+ */
+class Database : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Create a default database that uses JSON file for tuning records.
+   * \param record_path The path to the database file.
+   * \param workload_path The path to the workload registry file.
+   * \param allow_missing Whether to create new file when the given path is not found.
+   */
+  TVM_DLL static Database JSONFile(String record_path, String workload_path, bool allow_missing);

Review comment:
       Do we ever need to create a default database from scratch?

##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    v->Visit("shash", &shash);
+    // `token_id_` is not visited
+  }
+
+  /*!
+   * \brief Export the workload token to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.WorkloadToken";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadTokenNode, runtime::Object);
+};
+
+/*!
+ * \brief Managed reference to WorkloadTokenNode.
+ *  \sa WorkloadTokenNode
+ */
+class WorkloadToken : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Constructor of WorkloadToken.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   * \param token_id The workload's token id.
+   */
+  TVM_DLL explicit WorkloadToken(IRModule mod, String shash, int64_t token_id);
+
+  /*!
+   * \brief Create a workload token from a json object.
+   * \param json_obj The json object.
+   * \param token_id The workload's token id.
+   * \return The created workload token.
+   */
+  TVM_DLL static WorkloadToken FromJSON(const ObjectRef& json_obj, int64_t token_id);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(WorkloadToken, runtime::ObjectRef, WorkloadTokenNode);
+};
+
+/*! \brief The class for workload registry. */
+class WorkloadRegistryNode : public runtime::Object {
+ public:
+  /*! \brief The workload registry's storage path. */
+  String path;
+  /*! \brief The map from workload to its corresponding workload token id. */
+  std::unordered_map<IRModule, int64_t, tvm::StructuralHash, tvm::StructuralEqual> mod2token_id_;
+  /*! \brief The list of workload tokens. */
+  std::vector<WorkloadToken> workload_tokens_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("path", &path);
+    // `mod2token_id_` is not visited
+    // `workload_tokens_` is not visited

Review comment:
       Why not?

##########
File path: python/tvm/meta_schedule/database/json_file.py
##########
@@ -0,0 +1,37 @@
+# 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.
+"""The default database that uses a JSON File to store tuning records"""
+from tvm._ffi import register_object
+
+from .. import _ffi_api
+from .database import Database
+
+
+@register_object("meta_schedule.JSONFile")
+class JSONFile(Database):
+    def __init__(
+        self,
+        record_path: str,
+        workload_path: str,
+        allow_missing: bool = True,
+    ) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.DatabaseJSONFile,  # type: ignore # pylint: disable=no-member
+            record_path,
+            workload_path,
+            allow_missing,
+        )

Review comment:
       I'm not sure I grok the design of working directly with a JSON wrapper like this instead of with a Database?

##########
File path: src/meta_schedule/database/json_file.cc
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 <set>
+
+#include "../utils.h"
+#include "tvm/node/structural_equal.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The struct defining comparison function of sorting by mean run seconds. */
+struct SortByMeanRunSecs {

Review comment:
       Odd naming, but I don't have a better one.

##########
File path: python/tvm/meta_schedule/database/database.py
##########
@@ -0,0 +1,195 @@
+# 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.
+"""Tuning record database"""
+from typing import Any, List
+
+from tvm._ffi import register_object
+from tvm.ir.module import IRModule
+from tvm.runtime import Object
+from tvm.target import Target
+from tvm.tir.schedule import Trace
+
+from .. import _ffi_api
+from ..arg_info import ArgInfo
+from ..workload_registry import WorkloadRegistry, WorkloadToken
+
+
+@register_object("meta_schedule.TuningRecord")
+class TuningRecord(Object):
+    """
+    The class of tuning records.
+
+    Parameters
+    ----------
+    trace : tvm.ir.Trace
+        The trace of the tuning record.
+    run_secs : List[float]
+        The run time of the tuning record.
+    workload : WorkloadToken
+        The workload token of the tuning record.
+    target : Target
+        The target of the tuning record.
+    args_info : List[ArgInfo]
+        The argument information of the tuning record.
+    """
+
+    trace: Trace
+    run_secs: List[float]
+    workload: WorkloadToken
+    target: Target
+    args_info: List[ArgInfo]
+
+    def __init__(
+        self,
+        trace: Trace,
+        run_secs: List[float],
+        workload: WorkloadToken,
+        target: Target,
+        args_info: List[ArgInfo],
+    ) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningRecord,  # type: ignore # pylint: disable=no-member
+            trace,
+            run_secs,
+            workload,
+            target,
+            args_info,
+        )
+
+    def as_json(self) -> Any:
+        """Export the tuning record to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _ffi_api.TuningRecordAsJSON(self)  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any, reg: WorkloadRegistry) -> "TuningRecord":
+        """Create a tuning record from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+        reg : WorkloadRegistry
+            The workload registry.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.TuningRecordFromJSON(json_obj, reg)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.Database")
+class Database(Object):
+    """The abstract database interface."""
+
+    def add(self, record: TuningRecord) -> None:
+        """Add a tuning record to the database.
+
+        Parameters
+        ----------
+        record : TuningRecord
+            The tuning record to add.
+        """
+        _ffi_api.DatabaseAdd(self, record)  # type: ignore # pylint: disable=no-member
+
+    def get_top_k(self, workload: WorkloadToken, top_k: int) -> List[TuningRecord]:
+        """Get the top K tuning records of given workload from the database.
+
+        Parameters
+        ----------
+        workload : WorkloadToken
+            The workload to be searched for.
+        top_k : int
+            The number of top records to get.
+
+        Returns
+        -------
+        top_k_records : List[TuningRecord]
+            The top K records.
+        """
+        return _ffi_api.DatabaseGetTopK(self, workload, top_k)  # type: ignore # pylint: disable=no-member
+
+    def lookup_or_add(self, mod: IRModule) -> WorkloadToken:
+        """Look up or add workload to the database if missing.
+
+        Parameters
+        ----------
+        mod : IRModule
+            The IRModule to be searched for or added.
+
+        Returns
+        -------
+        workload : WorkloadToken
+            The workload token of the given IRModule.
+        """
+        return _ffi_api.DatabaseLookupOrAdd(self, mod)  # type: ignore # pylint: disable=no-member
+
+    def __len__(self) -> int:
+        """Get the number of records in the database.
+
+        Returns
+        -------
+        num_records : int
+        """
+        return _ffi_api.DatabaseSize(self)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.PyDatabase")
+class PyDatabase(Database):
+    """An abstract Database with customized methods on the python-side."""
+
+    def __init__(self):
+        """Constructor."""
+
+        def f_add(record: TuningRecord) -> None:
+            self.add(record)
+
+        def f_get_top_k(workload: WorkloadToken, top_k: int) -> List[TuningRecord]:
+            return self.get_top_k(workload, top_k)
+
+        def f_lookup_or_add(mod: IRModule) -> WorkloadToken:
+            return self.lookup_or_add(mod)
+
+        def f_size() -> int:
+            return len(self)
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.DatabasePyDatabase,  # pylint: disable=no-member
+            f_add,
+            f_get_top_k,
+            f_lookup_or_add,
+            f_size,
+        )
+
+    def add(self, record: TuningRecord) -> None:
+        raise NotImplementedError
+
+    def get_top_k(self, workload: WorkloadToken, top_k: int) -> List[TuningRecord]:
+        raise NotImplementedError
+
+    def lookup_or_add(self, mod: IRModule) -> WorkloadToken:
+        raise NotImplementedError
+
+    def __len__(self) -> int:
+        raise NotImplementedError

Review comment:
       :heart:  Thank you




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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3b] Database

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


   PR updated according to our discussion 👍  @mbrookhart @comaniac please take another look. Thanks guys for the discussion!


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

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

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



[GitHub] [tvm] mbrookhart commented on pull request #9061: [Meta Schedule][M3c] Database

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


   The first three points and the last one make sense, but I think I'm a little confused by `JSONFile` being the default user-facing database. JSON means a very particular thing, you convert to/from it, but it doesn't have class methods or a user interface, per say. I would say all of the JSON functionality is important for conversion and serialization, but I'm not sure it makes sense for it to be the primary interface point?


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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,210 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/meta_schedule/workload_registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload token. */
+  WorkloadToken workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, workload token id, serialized target, and
+   *  argument information.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, WorkloadToken workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param reg The workload registry.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const WorkloadRegistry& reg);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void Add(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const WorkloadToken& workload, int top_k) = 0;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  virtual WorkloadToken LookupOrAdd(const IRModule& mod) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `Add` method.
+   * \param record The tuning record to be added.
+   */
+  using FAdd = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const WorkloadToken&, int)>;
+  /*!
+   * \brief The function type of `LookupOrAdd` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload token of the given IRModule.
+   */
+  using FLookupOrAdd = runtime::TypedPackedFunc<WorkloadToken(const IRModule&)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `Add` function. */
+  FAdd f_add;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `LookupOrAdd` function. */
+  FLookupOrAdd f_lookup_or_add;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_add` is not visited
+    // `f_get_top_k` is not visited
+    // `f_lookup_or_add` is not visited
+    // `f_size` is not visited

Review comment:
       Yeah one way to provide access to these `PackedFunc`s is to add a few APIs each of which return a corresponding `PackedFunc` as results, because TVM FFI does support returning `PackedFunc`s.
   
   Right now I don't think we have such need to access these, because the python interpreter could just access their member functions `PyDatabase.XXX` directly without having to go through FFI
   




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

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

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



[GitHub] [tvm] junrushao1994 edited a comment on pull request #9061: [Meta Schedule][M3c] Database

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


   > It seems the direct use of the Database is all wrapped in a JSON-related file object, which kind of confuses me, would you mind providing a brief explanation of the class structure somewhere?
   
   Hey thanks @mbrookhart for asking! Let's focus on the database API and ignore the workload registry part (which will be removed according to previous discussion with Cody in this thread).
   
   * The base abstract class is `Database`, which has 4 pure virtual methods: `commit_workload`, `commit_tuning_record`, `get_top_k`, `size`
   * The system interacts with any subclass of `Database` by calling these 4 methods
   * To implement a concrete subclass of `Database`: in pure C++, we can just inherit Database and do so; in pure python, we provide `PyDatabase` which plays some tricks so that everything can be implemented in pure python
   * `JSONFile` is a simple default implementation of `Database` for open source


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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: include/tvm/meta_schedule/workload_registry.h
##########
@@ -0,0 +1,136 @@
+/*
+ * 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_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+#define TVM_META_SCHEDULE_WORKLOAD_REGISTRY_H_
+
+#include <tvm/ir/module.h>
+
+#include <unordered_map>
+#include <vector>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief The class of workload tokens. */
+class WorkloadTokenNode : public runtime::Object {
+ public:
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  String shash;
+  /*! \brief The workload's token id. */
+  int64_t token_id_;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    v->Visit("shash", &shash);
+    // `token_id_` is not visited
+  }
+
+  /*!
+   * \brief Export the workload token to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+
+  static constexpr const char* _type_key = "meta_schedule.WorkloadToken";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadTokenNode, runtime::Object);
+};
+
+/*!
+ * \brief Managed reference to WorkloadTokenNode.
+ *  \sa WorkloadTokenNode
+ */
+class WorkloadToken : public runtime::ObjectRef {
+ public:
+  /*!
+   * \brief Constructor of WorkloadToken.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   * \param token_id The workload's token id.
+   */
+  TVM_DLL explicit WorkloadToken(IRModule mod, String shash, int64_t token_id);
+
+  /*!
+   * \brief Create a workload token from a json object.
+   * \param json_obj The json object.
+   * \param token_id The workload's token id.
+   * \return The created workload token.
+   */
+  TVM_DLL static WorkloadToken FromJSON(const ObjectRef& json_obj, int64_t token_id);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(WorkloadToken, runtime::ObjectRef, WorkloadTokenNode);
+};
+
+/*! \brief The class for workload registry. */
+class WorkloadRegistryNode : public runtime::Object {

Review comment:
       No it's not a right name




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

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

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



[GitHub] [tvm] junrushao1994 merged pull request #9061: [Meta Schedule][M3b] Database

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


   


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

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

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



[GitHub] [tvm] mbrookhart commented on a change in pull request #9061: [Meta Schedule][M3b] Database

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



##########
File path: include/tvm/meta_schedule/database.h
##########
@@ -0,0 +1,275 @@
+/*
+ * 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_META_SCHEDULE_DATABASE_H_
+#define TVM_META_SCHEDULE_DATABASE_H_
+
+#include <tvm/meta_schedule/arg_info.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/schedule/trace.h>
+
+namespace tvm {
+namespace meta_schedule {
+
+/*! \brief A workload, i.e. an IRModule and its structural hash. */
+class WorkloadNode : public runtime::Object {
+ public:
+  /*! \brief The type of structural hash */
+  using THashCode = size_t;
+  /*! \brief The workload's IRModule. */
+  IRModule mod;
+  /*! \brief The workload's structural hash. */
+  THashCode shash;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("mod", &mod);
+    // `shash` is not visited because TVM FFI doesn't support uint64_t
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.Workload";
+  TVM_DECLARE_FINAL_OBJECT_INFO(WorkloadNode, runtime::Object);
+
+  /*!
+   * \brief Export the workload to a JSON string.
+   * \return An array containing the structural hash and the base64 json string.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief Managed reference to WorkloadNode.
+ *  \sa WorkloadNode
+ */
+class Workload : public runtime::ObjectRef {
+ public:
+  using THashCode = WorkloadNode::THashCode;
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   */
+  TVM_DLL explicit Workload(IRModule mod);
+  /*!
+   * \brief Constructor of Workload.
+   * \param mod The workload's IRModule.
+   * \param shash The workload's structural hash.
+   */
+  TVM_DLL explicit Workload(IRModule mod, THashCode shash);
+  /*!
+   * \brief Create a workload from a json object.
+   * \param json_obj The json object.
+   * \return The created workload.
+   */
+  TVM_DLL static Workload FromJSON(const ObjectRef& json_obj);
+
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(Workload, runtime::ObjectRef, WorkloadNode);
+};
+
+/*! \brief The hash method for Workload */
+struct WorkloadHash {
+  size_t operator()(const Workload& a) const { return a->shash; }
+};
+
+/*! \brief The equality check for Workload */
+struct WorkloadEqual {
+  bool operator()(const Workload& a, const Workload& b) const {
+    return a->shash == b->shash && tvm::StructuralEqual()(a->mod, b->mod);
+  }
+};
+
+/*! \brief The class of tuning records. */
+class TuningRecordNode : public runtime::Object {
+ public:
+  /*! \brief The trace tuned. */
+  tir::Trace trace;
+  /*! \brief The profiling result in seconds. */
+  Array<FloatImm> run_secs;
+  /*! \brief The workload. */
+  Workload workload{nullptr};
+  /*! \brief The target for tuning. */
+  Target target;
+  /*! \brief The argument information. */
+  Array<ArgInfo> args_info;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("trace", &trace);
+    v->Visit("run_secs", &run_secs);
+    v->Visit("workload", &workload);
+    v->Visit("target", &target);
+    v->Visit("args_info", &args_info);
+  }
+
+  static constexpr const char* _type_key = "meta_schedule.TuningRecord";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningRecordNode, runtime::Object);
+
+  /*!
+   * \brief Export the tuning record to a JSON string.
+   * \return An array containing the trace, running secs, serialized target, and
+   * argument information.
+   */
+  ObjectRef AsJSON() const;
+};
+
+/*!
+ * \brief The managed reference of TuningRecordNode.
+ * \sa TuningRecordNode
+ */
+class TuningRecord : public runtime::ObjectRef {
+ public:
+  /*!
+   \brief Constructor of a tuning record.
+   \param trace The trace of the tuning record.
+   \param run_secs The running time of the tuning record.
+   \param workload The workload of the tuning record.
+   \param target The target of the tuning record.
+   \param args_info The argument information of the tuning record.
+  */
+  TVM_DLL explicit TuningRecord(tir::Trace trace, Array<FloatImm> run_secs, Workload workload,
+                                Target target, Array<ArgInfo> args_info);
+  /*!
+   * \brief Create a tuning record from a json object.
+   * \param json_obj The json object.
+   * \param workload The workload.
+   * \return The tuning record created.
+   */
+  TVM_DLL static TuningRecord FromJSON(const ObjectRef& json_obj, const Workload& workload);
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TuningRecord, runtime::ObjectRef, TuningRecordNode);
+};
+
+/* \brief The abstract interface of database. */
+class DatabaseNode : public runtime::Object {
+ public:
+  /*! \brief Default destructor */
+  virtual ~DatabaseNode() = default;
+  /*!
+   * \brief Look up or add workload to the database if missing.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  virtual Workload CommitWorkload(const IRModule& mod) = 0;
+  /*!
+   * \brief Add a tuning record to the database.
+   * \param record The tuning record to be added.
+   */
+  virtual void CommitTuningRecord(const TuningRecord& record) = 0;
+  /*!
+   * \brief Get the top K tuning records of given workload from the database.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  virtual Array<TuningRecord> GetTopK(const Workload& workload, int top_k) = 0;
+  /*!
+   * \brief Get the size of the database.
+   * \return The size of the database.
+   */
+  virtual int64_t Size() = 0;
+
+  static constexpr const char* _type_key = "meta_schedule.Database";
+  TVM_DECLARE_BASE_OBJECT_INFO(DatabaseNode, runtime::Object);
+};
+
+/*! \brief The database with customized methods on the python-side. */
+class PyDatabaseNode : public DatabaseNode {
+ public:
+  /*!
+   * \brief The function type of `CommitWorkload` method.
+   * \param mod The IRModule to be searched for or added.
+   * \return The workload corresponding to the given IRModule.
+   */
+  using FCommitWorkload = runtime::TypedPackedFunc<Workload(const IRModule&)>;
+  /*!
+   * \brief The function type of `CommitTuningRecord` method.
+   * \param record The tuning record to be added.
+   */
+  using FCommitTuningRecord = runtime::TypedPackedFunc<void(const TuningRecord&)>;
+  /*!
+   * \brief The function type of `GetTopK` method.
+   * \param workload The workload to be searched for.
+   * \param top_k The number of top records to be returned.
+   * \return An array of top K tuning records for the given workload.
+   */
+  using FGetTopK = runtime::TypedPackedFunc<Array<TuningRecord>(const Workload&, int)>;
+  /*!
+   * \brief The function type of `Size` method.
+   * \return The size of the database.
+   */
+  using FSize = runtime::TypedPackedFunc<int64_t()>;
+
+  /*! \brief The packed function to the `CommitWorkload` function. */
+  FCommitWorkload f_commit_workload;
+  /*! \brief The packed function to the `CommitTuningRecord` function. */
+  FCommitTuningRecord f_commit_tuning_record;
+  /*! \brief The packed function to the `GetTopK` function. */
+  FGetTopK f_get_top_k;
+  /*! \brief The packed function to the `Size` function. */
+  FSize f_size;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    // `f_commit_workload` is not visited

Review comment:
       Nit: Do we need to define this function if it's empty?
   If so, add a comment that packed funcs can't be passed back through the FFI




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

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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #9061: [Meta Schedule][M3c] Database

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



##########
File path: python/tvm/meta_schedule/database/database.py
##########
@@ -0,0 +1,240 @@
+# 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.
+"""Tuning record database"""
+from typing import Any, List
+
+from tvm._ffi import register_object
+from tvm.ir.module import IRModule
+from tvm.runtime import Object
+from tvm.target import Target
+from tvm.tir.schedule import Trace
+
+from .. import _ffi_api
+from ..arg_info import ArgInfo
+from ..utils import _json_de_tvm
+
+
+@register_object("meta_schedule.Workload")
+class Workload(Object):
+    """A workload, i.e. an IRModule and its structural hash.
+
+    Parameters
+    ----------
+    mod : IRModule
+        The workload's IRModule
+    """
+
+    mod: IRModule
+
+    def __init__(self, mod: IRModule) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.Workload,  # type: ignore # pylint: disable=no-member
+            mod,
+        )
+
+    def as_json(self) -> Any:
+        """Export the workload to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _json_de_tvm(_ffi_api.WorkloadAsJSON(self))  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any) -> "Workload":
+        """Create a workload from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.WorkloadFromJSON(json_obj)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.TuningRecord")
+class TuningRecord(Object):
+    """The class of tuning records.
+
+    Parameters
+    ----------
+    trace : tvm.ir.Trace
+        The trace of the tuning record.
+    run_secs : List[float]
+        The run time of the tuning record.
+    workload : Workload
+        The workload of the tuning record.
+    target : Target
+        The target of the tuning record.
+    args_info : List[ArgInfo]
+        The argument information of the tuning record.
+    """
+
+    trace: Trace
+    run_secs: List[float]
+    workload: Workload
+    target: Target
+    args_info: List[ArgInfo]
+
+    def __init__(
+        self,
+        trace: Trace,
+        run_secs: List[float],
+        workload: Workload,
+        target: Target,
+        args_info: List[ArgInfo],
+    ) -> None:
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningRecord,  # type: ignore # pylint: disable=no-member
+            trace,
+            run_secs,
+            workload,
+            target,
+            args_info,
+        )
+
+    def as_json(self) -> Any:
+        """Export the tuning record to a JSON string.
+
+        Returns
+        -------
+        json_str : str
+            The JSON string exported.
+        """
+        return _json_de_tvm(_ffi_api.TuningRecordAsJSON(self))  # type: ignore # pylint: disable=no-member
+
+    @staticmethod
+    def from_json(json_obj: Any, workload: Workload) -> "TuningRecord":
+        """Create a tuning record from a json object.
+
+        Parameters
+        ----------
+        json_obj : Any
+            The json object to parse.
+        workload : Workload
+            The workload.
+
+        Returns
+        -------
+        tuning_record : TuningRecord
+            The parsed tuning record.
+        """
+        return _ffi_api.TuningRecordFromJSON(json_obj, workload)  # type: ignore # pylint: disable=no-member
+
+
+@register_object("meta_schedule.Database")
+class Database(Object):
+    """The abstract database interface."""
+
+    def commit_workload(self, mod: IRModule) -> Workload:
+        """Look up or add workload to the database if missing.
+
+        Parameters
+        ----------
+        mod : IRModule
+            The IRModule to be searched for or added.
+
+        Returns
+        -------
+        workload : Workload
+            The workload corresponding to the given IRModule.
+        """
+        return _ffi_api.DatabaseCommitWorkload(self, mod)  # type: ignore # pylint: disable=no-member
+
+    def commit_tuning_record(self, record: TuningRecord) -> None:
+        """Add a tuning record to the database.

Review comment:
       We don't have deduplication mechanism though. It's simply appending a row to the end of the database




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

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

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



[GitHub] [tvm] junrushao1994 commented on pull request #9061: [Meta Schedule][M3c] Database

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


   CC @comaniac @mbrookhart @jroesch @ZihengJiang 


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