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/23 21:24:52 UTC

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

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