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 2022/09/29 23:05:11 UTC

[GitHub] [tvm] zxybazh opened a new pull request, #12948: [MetaSchedule] Add JSON Database Validation Scripts

zxybazh opened a new pull request, #12948:
URL: https://github.com/apache/tvm/pull/12948

   This PR introduces a validation script to check result accuracy between the scheduled IRModules and original IRModule stored in MetaSchedule database. The validate function could also be reused for other type of databases in MetaSchedule. The result would be printed out on the screen as validation passed or failed at some records.


-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018265616


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--baseline-target",
+        type=str,
+        default="llvm -num-cores=1",
+        required=False,
+        help="The baseline target to compile the original module.",
+    )
+    args.add_argument(
+        "--rpc-host",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-port",
+        type=int,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-key",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--number",
+        type=int,
+        default=3,
+    )
+    args.add_argument(
+        "--repeat",
+        type=int,
+        default=1,
+    )
+    args.add_argument(
+        "--min-repeat-ms",
+        type=int,
+        default=100,
+    )
+    args.add_argument(
+        "--cpu-flush",
+        type=lambda x: bool(strtobool(x)),
+        help="example: True / False",
+        required=True,
+    )
+    parsed = args.parse_args()
+    parsed.target = tvm.target.Target(parsed.target)
+    parsed.rpc_config = ms.runner.RPCConfig(
+        tracker_host=parsed.rpc_host,
+        tracker_port=parsed.rpc_port,
+        tracker_key=parsed.rpc_key,
+        session_timeout_sec=600,
+    )
+    if parsed.cpu_flush and parsed.target.kind.name != "llvm":
+        warnings.warn("cpu_flush is only supported on llvm target")
+    return parsed
+
+
+# logging
+logging.basicConfig(
+    format="%(asctime)s.%(msecs)03d %(levelname)s %(message)s", datefmt="%Y-%m-%d %H:%M:%S"
+)
+logging.getLogger("tvm.meta_schedule").setLevel(logging.INFO)
+
+# arg parser
+ARGS = _parse_args()
+
+
+@register_func("tvm.meta_schedule.testing.default_input_generator")
+def default_input_generator(mod: IRModule) -> List[tvm.nd.NDArray]:
+    args_info = ms.arg_info.TensorInfo.from_prim_func(mod["main"])
+    inputs = [
+        tvm.nd.array(generate_input_data(input_shape=arg_info.shape, input_dtype=arg_info.dtype))
+        for arg_info in args_info
+    ]
+    return inputs
+
+
+@register_func("tvm.meta_schedule.testing.default_check_metric")
+def default_check_metric(a: List[tvm.nd.NDArray], b: List[tvm.nd.NDArray]) -> bool:
+    assert len(a) == len(b), "Different number of outputs from two modules"
+    for i, _ in enumerate(a):
+        if not np.allclose(a[i].numpy(), b[i].numpy(), rtol=1e-3, atol=2e-3):
+            return False
+    return True
+
+
+def validate_correctness(
+    original_mod: IRModule,  # compiled for "baseline_target"
+    scheduled_mod: IRModule,  # compiled for "target"
+    *,
+    baseline_target: Union[str, Target],
+    target: Union[str, Target],
+    dev_type: str,
+    rpc_config: ms.runner.RPCConfig,
+    f_input_generator: Union[
+        str, Callable[[IRModule], List[tvm.nd.NDArray]]
+    ] = default_input_generator,
+    f_check_metric: Union[
+        str, Callable[[tvm.nd.NDArray, tvm.nd.NDArray], bool]
+    ] = default_check_metric,
+) -> bool:
+    """Function to validate the correctness of a scheduled module.
+
+    Parameters
+    ----------
+    original_mod : IRModule
+        The original module to be compiled.
+    scheduled_mod : IRModule
+        The scheduled module to be compiled.
+    target : Target
+        The target to compile the scheduled module.
+    rpc_config : RPCConfig
+        The RPCConfig to run the scheduled module.
+    f_input_generator : Union[str, Callable]
+        The function to generate the input data.
+    f_check_metric : Union[str, Callable]
+        The function to check the metric.
+
+    Returns
+    -------
+    result : ...
+        The result of the validation.
+    """
+
+    def to_numpy(a: List[tvm.nd.NDArray]) -> List[np.ndarray]:
+        """Convert a list of TVM NDArray to a list of numpy array"""
+        assert a is not None, "Empty result cannot be converted to numpy"
+        return [x.numpy() for x in a]
+
+    def to_tvm_ndarray(a: List[np.ndarray]) -> List[tvm.nd.NDArray]:
+        """Convert a list of numpy array to a list of TVM NDArray"""
+        assert a is not None, "Empty result cannot be converted to TVM NDArray"
+        return [tvm.nd.array(x) for x in a]
+
+    def build_and_run(mod: IRModule, target: Target, dev_type: str) -> np.ndarray:
+        """Build and run the module on the target device."""
+        rt_mod = tvm.build(mod, target=target)
+        return run_module_via_rpc(
+            rpc_config=rpc_config,
+            lib=rt_mod,
+            dev_type=dev_type,
+            args={i: v for i, v in enumerate(inputs)},  # pylint: disable=unnecessary-comprehension
+            continuation=create_computer(backend="tir"),
+            backend="tir",
+        )
+
+    # make targets
+    target = Target(target)
+    baseline_target = Target(baseline_target)
+    # fetch functions & prepare inputs
+    if isinstance(f_input_generator, str):
+        f_input_generator = get_global_func(f_input_generator)
+    if isinstance(f_check_metric, str):
+        f_check_metric = get_global_func(f_check_metric)
+    inputs = to_numpy(f_input_generator(original_mod))  # type: ignore
+    # build & run original result
+    original_res = to_numpy(build_and_run(original_mod, target=baseline_target, dev_type="cpu"))
+    scheduled_res = to_numpy(build_and_run(scheduled_mod, target=target, dev_type=dev_type))
+    # check metric
+    if f_check_metric(to_tvm_ndarray(original_res), to_tvm_ndarray(scheduled_res)):  # type: ignore
+        return True
+    else:
+        print(
+            ("\n\n").join(
+                [
+                    "Validation failed!",
+                    "Original Result:" + DELIMITOR + str(original_res),
+                    "Scheduled Result:" + DELIMITOR + str(scheduled_res),
+                    "Input:" + DELIMITOR + str(inputs),
+                    "Original IRModule:" + DELIMITOR + original_mod.script(),
+                    "Scheduled IRModule:" + DELIMITOR + scheduled_mod.script(),
+                ]
+            )
+        )
+        return False
+
+
+def main():
+    """Main function"""
+    describe()
+    database = ms.database.JSONDatabase(
+        path_workload=ARGS.path_workload, path_tuning_record=ARGS.path_tuning_record
+    )

Review Comment:
   ```suggestion
       database = ms.database.create(work_dir=work_dir)
   ```



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018260802


##########
python/tvm/meta_schedule/testing/custom_builder_runner.py:
##########
@@ -143,7 +143,7 @@ def run_module_via_rpc(
     rpc_config: "RPCConfig",
     lib: Union["Module", "Executable"],
     dev_type: str,
-    args: Dict[str, "np.ndarray"],
+    args: Dict[Any, "np.ndarray"],

Review Comment:
   ```suggestion
       args: Dict[Union[int, str], "np.ndarray"],
   ```



##########
python/tvm/meta_schedule/testing/custom_builder_runner.py:
##########
@@ -17,7 +17,7 @@
 """Customized builder and runner methods"""
 # pylint: disable=import-outside-toplevel
 
-from typing import TYPE_CHECKING, Callable, Dict, List, Optional, Union
+from typing import TYPE_CHECKING, Dict, List, Any, Optional, Union, Callable

Review Comment:
   remove `Callable`?



-- 
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 diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
zxybazh commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018321243


##########
python/tvm/meta_schedule/testing/custom_builder_runner.py:
##########
@@ -17,7 +17,7 @@
 """Customized builder and runner methods"""
 # pylint: disable=import-outside-toplevel
 
-from typing import TYPE_CHECKING, Callable, Dict, List, Optional, Union
+from typing import TYPE_CHECKING, Dict, List, Any, Optional, Union, Callable

Review Comment:
   It's used by argument `continuation` in `run_module_via_rpc`.



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018264199


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--baseline-target",
+        type=str,
+        default="llvm -num-cores=1",
+        required=False,
+        help="The baseline target to compile the original module.",
+    )
+    args.add_argument(
+        "--rpc-host",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-port",
+        type=int,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-key",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--number",
+        type=int,
+        default=3,
+    )
+    args.add_argument(
+        "--repeat",
+        type=int,
+        default=1,
+    )
+    args.add_argument(
+        "--min-repeat-ms",
+        type=int,
+        default=100,
+    )
+    args.add_argument(
+        "--cpu-flush",
+        type=lambda x: bool(strtobool(x)),
+        help="example: True / False",
+        required=True,
+    )
+    parsed = args.parse_args()
+    parsed.target = tvm.target.Target(parsed.target)
+    parsed.rpc_config = ms.runner.RPCConfig(
+        tracker_host=parsed.rpc_host,
+        tracker_port=parsed.rpc_port,
+        tracker_key=parsed.rpc_key,
+        session_timeout_sec=600,
+    )
+    if parsed.cpu_flush and parsed.target.kind.name != "llvm":
+        warnings.warn("cpu_flush is only supported on llvm target")
+    return parsed
+
+
+# logging
+logging.basicConfig(
+    format="%(asctime)s.%(msecs)03d %(levelname)s %(message)s", datefmt="%Y-%m-%d %H:%M:%S"
+)
+logging.getLogger("tvm.meta_schedule").setLevel(logging.INFO)

Review Comment:
   ```suggestion
   logging.getLogger("tvm.meta_schedule").setLevel(logging.DEBUG)
   ```



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1002101759


##########
python/tvm/meta_schedule/testing/tune_utils.py:
##########
@@ -192,3 +192,50 @@ def f_time_per_layer(
             )
 
     return f_time_per_layer
+
+
+def create_computer(backend: str) -> Callable:

Review Comment:
   computer sounds like a weird name, do you have any better proposal?



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018262909


##########
python/tvm/meta_schedule/testing/tune_utils.py:
##########
@@ -192,3 +192,50 @@ def f_time_per_layer(
             )
 
     return f_time_per_layer
+
+
+def create_computer(backend: str) -> Callable:
+    """Create a function to fetch the computing result of running the given runtime module.
+
+    Parameters
+    ----------
+    backend : str
+        The backend to use, graph / vm.

Review Comment:
   ```suggestion
           The backend to use, graph / vm / tir. Only tir is supported for now.
   ```



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018340746


##########
python/tvm/meta_schedule/testing/custom_builder_runner.py:
##########
@@ -17,7 +17,7 @@
 """Customized builder and runner methods"""
 # pylint: disable=import-outside-toplevel
 
-from typing import TYPE_CHECKING, Callable, Dict, List, Optional, Union
+from typing import TYPE_CHECKING, Dict, List, Any, Optional, Union, Callable

Review Comment:
   sounds good. thanks for pointing this out!



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018271810


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--baseline-target",
+        type=str,
+        default="llvm -num-cores=1",
+        required=False,
+        help="The baseline target to compile the original module.",
+    )
+    args.add_argument(
+        "--rpc-host",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-port",
+        type=int,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-key",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--number",
+        type=int,
+        default=3,
+    )
+    args.add_argument(
+        "--repeat",
+        type=int,
+        default=1,
+    )
+    args.add_argument(
+        "--min-repeat-ms",
+        type=int,
+        default=100,
+    )
+    args.add_argument(
+        "--cpu-flush",
+        type=lambda x: bool(strtobool(x)),
+        help="example: True / False",
+        required=True,
+    )
+    parsed = args.parse_args()
+    parsed.target = tvm.target.Target(parsed.target)
+    parsed.rpc_config = ms.runner.RPCConfig(
+        tracker_host=parsed.rpc_host,
+        tracker_port=parsed.rpc_port,
+        tracker_key=parsed.rpc_key,
+        session_timeout_sec=600,
+    )
+    if parsed.cpu_flush and parsed.target.kind.name != "llvm":
+        warnings.warn("cpu_flush is only supported on llvm target")
+    return parsed
+
+
+# logging
+logging.basicConfig(
+    format="%(asctime)s.%(msecs)03d %(levelname)s %(message)s", datefmt="%Y-%m-%d %H:%M:%S"
+)
+logging.getLogger("tvm.meta_schedule").setLevel(logging.INFO)
+
+# arg parser
+ARGS = _parse_args()
+
+
+@register_func("tvm.meta_schedule.testing.default_input_generator")
+def default_input_generator(mod: IRModule) -> List[tvm.nd.NDArray]:
+    args_info = ms.arg_info.TensorInfo.from_prim_func(mod["main"])
+    inputs = [
+        tvm.nd.array(generate_input_data(input_shape=arg_info.shape, input_dtype=arg_info.dtype))
+        for arg_info in args_info
+    ]
+    return inputs
+
+
+@register_func("tvm.meta_schedule.testing.default_check_metric")
+def default_check_metric(a: List[tvm.nd.NDArray], b: List[tvm.nd.NDArray]) -> bool:
+    assert len(a) == len(b), "Different number of outputs from two modules"
+    for i, _ in enumerate(a):
+        if not np.allclose(a[i].numpy(), b[i].numpy(), rtol=1e-3, atol=2e-3):
+            return False
+    return True
+
+
+def validate_correctness(
+    original_mod: IRModule,  # compiled for "baseline_target"
+    scheduled_mod: IRModule,  # compiled for "target"
+    *,
+    baseline_target: Union[str, Target],
+    target: Union[str, Target],

Review Comment:
   ```suggestion
       baseline_target: Target,
       target: Target,
   ```



-- 
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] tvm-bot commented on pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
tvm-bot commented on PR #12948:
URL: https://github.com/apache/tvm/pull/12948#issuecomment-1275504151

   <!---bot-comment-->
   
   Thanks for contributing to TVM! Please refer to the contributing guidelines https://tvm.apache.org/docs/contribute/ for useful information and tips. Please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @-ing them in a comment.
   
   <!--bot-comment-docs-start-->
    * Built docs for commit de13c2b8f13e12540d6eabdbef5e38c66c2d43f9 can be found [here](https://pr-docs.tlcpack.ai/PR-12948/9/docs/index.html).<!--bot-comment-docs-end-->
   
   <sub>Generated by [tvm-bot](https://github.com/apache/tvm/blob/main/ci/README.md#github-actions)</sub>


-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018271136


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--baseline-target",
+        type=str,
+        default="llvm -num-cores=1",
+        required=False,
+        help="The baseline target to compile the original module.",
+    )
+    args.add_argument(
+        "--rpc-host",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-port",
+        type=int,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-key",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--number",
+        type=int,
+        default=3,
+    )
+    args.add_argument(
+        "--repeat",
+        type=int,
+        default=1,
+    )
+    args.add_argument(
+        "--min-repeat-ms",
+        type=int,
+        default=100,
+    )
+    args.add_argument(
+        "--cpu-flush",
+        type=lambda x: bool(strtobool(x)),
+        help="example: True / False",
+        required=True,
+    )
+    parsed = args.parse_args()
+    parsed.target = tvm.target.Target(parsed.target)
+    parsed.rpc_config = ms.runner.RPCConfig(
+        tracker_host=parsed.rpc_host,
+        tracker_port=parsed.rpc_port,
+        tracker_key=parsed.rpc_key,
+        session_timeout_sec=600,
+    )
+    if parsed.cpu_flush and parsed.target.kind.name != "llvm":
+        warnings.warn("cpu_flush is only supported on llvm target")
+    return parsed
+
+
+# logging
+logging.basicConfig(
+    format="%(asctime)s.%(msecs)03d %(levelname)s %(message)s", datefmt="%Y-%m-%d %H:%M:%S"
+)
+logging.getLogger("tvm.meta_schedule").setLevel(logging.INFO)
+
+# arg parser
+ARGS = _parse_args()
+
+
+@register_func("tvm.meta_schedule.testing.default_input_generator")
+def default_input_generator(mod: IRModule) -> List[tvm.nd.NDArray]:
+    args_info = ms.arg_info.TensorInfo.from_prim_func(mod["main"])
+    inputs = [
+        tvm.nd.array(generate_input_data(input_shape=arg_info.shape, input_dtype=arg_info.dtype))
+        for arg_info in args_info
+    ]
+    return inputs
+
+
+@register_func("tvm.meta_schedule.testing.default_check_metric")
+def default_check_metric(a: List[tvm.nd.NDArray], b: List[tvm.nd.NDArray]) -> bool:
+    assert len(a) == len(b), "Different number of outputs from two modules"
+    for i, _ in enumerate(a):
+        if not np.allclose(a[i].numpy(), b[i].numpy(), rtol=1e-3, atol=2e-3):
+            return False
+    return True
+
+
+def validate_correctness(
+    original_mod: IRModule,  # compiled for "baseline_target"
+    scheduled_mod: IRModule,  # compiled for "target"
+    *,
+    baseline_target: Union[str, Target],
+    target: Union[str, Target],
+    dev_type: str,
+    rpc_config: ms.runner.RPCConfig,
+    f_input_generator: Union[
+        str, Callable[[IRModule], List[tvm.nd.NDArray]]
+    ] = default_input_generator,
+    f_check_metric: Union[
+        str, Callable[[tvm.nd.NDArray, tvm.nd.NDArray], bool]
+    ] = default_check_metric,
+) -> bool:
+    """Function to validate the correctness of a scheduled module.
+
+    Parameters
+    ----------
+    original_mod : IRModule
+        The original module to be compiled.
+    scheduled_mod : IRModule
+        The scheduled module to be compiled.
+    target : Target
+        The target to compile the scheduled module.
+    rpc_config : RPCConfig
+        The RPCConfig to run the scheduled module.
+    f_input_generator : Union[str, Callable]
+        The function to generate the input data.
+    f_check_metric : Union[str, Callable]
+        The function to check the metric.
+
+    Returns
+    -------
+    result : ...
+        The result of the validation.
+    """
+
+    def to_numpy(a: List[tvm.nd.NDArray]) -> List[np.ndarray]:
+        """Convert a list of TVM NDArray to a list of numpy array"""
+        assert a is not None, "Empty result cannot be converted to numpy"
+        return [x.numpy() for x in a]
+
+    def to_tvm_ndarray(a: List[np.ndarray]) -> List[tvm.nd.NDArray]:
+        """Convert a list of numpy array to a list of TVM NDArray"""
+        assert a is not None, "Empty result cannot be converted to TVM NDArray"
+        return [tvm.nd.array(x) for x in a]
+
+    def build_and_run(mod: IRModule, target: Target, dev_type: str) -> np.ndarray:
+        """Build and run the module on the target device."""
+        rt_mod = tvm.build(mod, target=target)
+        return run_module_via_rpc(
+            rpc_config=rpc_config,
+            lib=rt_mod,
+            dev_type=dev_type,
+            args={i: v for i, v in enumerate(inputs)},  # pylint: disable=unnecessary-comprehension
+            continuation=create_computer(backend="tir"),
+            backend="tir",
+        )
+
+    # make targets
+    target = Target(target)
+    baseline_target = Target(baseline_target)
+    # fetch functions & prepare inputs
+    if isinstance(f_input_generator, str):
+        f_input_generator = get_global_func(f_input_generator)
+    if isinstance(f_check_metric, str):
+        f_check_metric = get_global_func(f_check_metric)
+    inputs = to_numpy(f_input_generator(original_mod))  # type: ignore
+    # build & run original result
+    original_res = to_numpy(build_and_run(original_mod, target=baseline_target, dev_type="cpu"))
+    scheduled_res = to_numpy(build_and_run(scheduled_mod, target=target, dev_type=dev_type))
+    # check metric
+    if f_check_metric(to_tvm_ndarray(original_res), to_tvm_ndarray(scheduled_res)):  # type: ignore
+        return True
+    else:
+        print(
+            ("\n\n").join(
+                [
+                    "Validation failed!",
+                    "Original Result:" + DELIMITOR + str(original_res),
+                    "Scheduled Result:" + DELIMITOR + str(scheduled_res),
+                    "Input:" + DELIMITOR + str(inputs),
+                    "Original IRModule:" + DELIMITOR + original_mod.script(),
+                    "Scheduled IRModule:" + DELIMITOR + scheduled_mod.script(),
+                ]
+            )
+        )
+        return False
+
+
+def main():
+    """Main function"""
+    describe()
+    database = ms.database.JSONDatabase(
+        path_workload=ARGS.path_workload, path_tuning_record=ARGS.path_tuning_record
+    )
+    assert Target(ARGS.target).kind.name in ["llvm", "cuda"]
+    dev_type = "cpu" if Target(ARGS.target).kind.name == "llvm" else "cuda"
+    records = database.get_all_tuning_records()
+    with ms.Profiler() as profiler:
+        for i, record in enumerate(records):
+            scope_name = f"validate #{i}"
+            with profiler.timeit(scope_name):
+                original_mod = record.workload.mod
+                sch = Schedule(original_mod)
+                record.trace.apply_to_schedule(sch=sch, remove_postproc=False)
+                scheduled_mod = sch.mod
+                flag = False

Review Comment:
   ```suggestion
                   is_success = False
   ```



-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018264754


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--baseline-target",
+        type=str,
+        default="llvm -num-cores=1",
+        required=False,
+        help="The baseline target to compile the original module.",
+    )
+    args.add_argument(
+        "--rpc-host",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-port",
+        type=int,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-key",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--number",
+        type=int,
+        default=3,
+    )
+    args.add_argument(
+        "--repeat",
+        type=int,
+        default=1,
+    )
+    args.add_argument(
+        "--min-repeat-ms",
+        type=int,
+        default=100,
+    )
+    args.add_argument(
+        "--cpu-flush",
+        type=lambda x: bool(strtobool(x)),
+        help="example: True / False",
+        required=True,
+    )
+    parsed = args.parse_args()
+    parsed.target = tvm.target.Target(parsed.target)
+    parsed.rpc_config = ms.runner.RPCConfig(
+        tracker_host=parsed.rpc_host,
+        tracker_port=parsed.rpc_port,
+        tracker_key=parsed.rpc_key,
+        session_timeout_sec=600,
+    )
+    if parsed.cpu_flush and parsed.target.kind.name != "llvm":
+        warnings.warn("cpu_flush is only supported on llvm target")
+    return parsed
+
+
+# logging
+logging.basicConfig(
+    format="%(asctime)s.%(msecs)03d %(levelname)s %(message)s", datefmt="%Y-%m-%d %H:%M:%S"
+)
+logging.getLogger("tvm.meta_schedule").setLevel(logging.INFO)

Review Comment:
   ```suggestion
   logging.getLogger("tvm.meta_schedule").setLevel(logging.DEBUG)
   ```



-- 
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 diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
zxybazh commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1002123131


##########
python/tvm/meta_schedule/testing/tune_utils.py:
##########
@@ -192,3 +192,50 @@ def f_time_per_layer(
             )
 
     return f_time_per_layer
+
+
+def create_computer(backend: str) -> Callable:

Review Comment:
   What about `calculator`? The point here is to show this function computes the results for the given workload.



-- 
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 merged pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
zxybazh merged PR #12948:
URL: https://github.com/apache/tvm/pull/12948


-- 
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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018269279


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--baseline-target",
+        type=str,
+        default="llvm -num-cores=1",
+        required=False,
+        help="The baseline target to compile the original module.",
+    )
+    args.add_argument(
+        "--rpc-host",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-port",
+        type=int,
+        required=True,
+    )
+    args.add_argument(
+        "--rpc-key",
+        type=str,
+        required=True,
+    )
+    args.add_argument(
+        "--number",
+        type=int,
+        default=3,
+    )
+    args.add_argument(
+        "--repeat",
+        type=int,
+        default=1,
+    )
+    args.add_argument(
+        "--min-repeat-ms",
+        type=int,
+        default=100,
+    )
+    args.add_argument(
+        "--cpu-flush",
+        type=lambda x: bool(strtobool(x)),
+        help="example: True / False",
+        required=True,
+    )
+    parsed = args.parse_args()
+    parsed.target = tvm.target.Target(parsed.target)
+    parsed.rpc_config = ms.runner.RPCConfig(
+        tracker_host=parsed.rpc_host,
+        tracker_port=parsed.rpc_port,
+        tracker_key=parsed.rpc_key,
+        session_timeout_sec=600,
+    )
+    if parsed.cpu_flush and parsed.target.kind.name != "llvm":
+        warnings.warn("cpu_flush is only supported on llvm target")
+    return parsed
+
+
+# logging
+logging.basicConfig(
+    format="%(asctime)s.%(msecs)03d %(levelname)s %(message)s", datefmt="%Y-%m-%d %H:%M:%S"
+)
+logging.getLogger("tvm.meta_schedule").setLevel(logging.INFO)
+
+# arg parser
+ARGS = _parse_args()
+
+
+@register_func("tvm.meta_schedule.testing.default_input_generator")
+def default_input_generator(mod: IRModule) -> List[tvm.nd.NDArray]:
+    args_info = ms.arg_info.TensorInfo.from_prim_func(mod["main"])
+    inputs = [
+        tvm.nd.array(generate_input_data(input_shape=arg_info.shape, input_dtype=arg_info.dtype))
+        for arg_info in args_info
+    ]
+    return inputs
+
+
+@register_func("tvm.meta_schedule.testing.default_check_metric")
+def default_check_metric(a: List[tvm.nd.NDArray], b: List[tvm.nd.NDArray]) -> bool:
+    assert len(a) == len(b), "Different number of outputs from two modules"
+    for i, _ in enumerate(a):
+        if not np.allclose(a[i].numpy(), b[i].numpy(), rtol=1e-3, atol=2e-3):
+            return False
+    return True
+
+
+def validate_correctness(
+    original_mod: IRModule,  # compiled for "baseline_target"
+    scheduled_mod: IRModule,  # compiled for "target"
+    *,
+    baseline_target: Union[str, Target],
+    target: Union[str, Target],
+    dev_type: str,
+    rpc_config: ms.runner.RPCConfig,
+    f_input_generator: Union[
+        str, Callable[[IRModule], List[tvm.nd.NDArray]]
+    ] = default_input_generator,
+    f_check_metric: Union[
+        str, Callable[[tvm.nd.NDArray, tvm.nd.NDArray], bool]
+    ] = default_check_metric,
+) -> bool:
+    """Function to validate the correctness of a scheduled module.
+
+    Parameters
+    ----------
+    original_mod : IRModule
+        The original module to be compiled.
+    scheduled_mod : IRModule
+        The scheduled module to be compiled.
+    target : Target
+        The target to compile the scheduled module.
+    rpc_config : RPCConfig
+        The RPCConfig to run the scheduled module.
+    f_input_generator : Union[str, Callable]
+        The function to generate the input data.
+    f_check_metric : Union[str, Callable]
+        The function to check the metric.
+
+    Returns
+    -------
+    result : ...
+        The result of the validation.
+    """
+
+    def to_numpy(a: List[tvm.nd.NDArray]) -> List[np.ndarray]:
+        """Convert a list of TVM NDArray to a list of numpy array"""
+        assert a is not None, "Empty result cannot be converted to numpy"
+        return [x.numpy() for x in a]
+
+    def to_tvm_ndarray(a: List[np.ndarray]) -> List[tvm.nd.NDArray]:
+        """Convert a list of numpy array to a list of TVM NDArray"""
+        assert a is not None, "Empty result cannot be converted to TVM NDArray"
+        return [tvm.nd.array(x) for x in a]
+
+    def build_and_run(mod: IRModule, target: Target, dev_type: str) -> np.ndarray:
+        """Build and run the module on the target device."""
+        rt_mod = tvm.build(mod, target=target)
+        return run_module_via_rpc(
+            rpc_config=rpc_config,
+            lib=rt_mod,
+            dev_type=dev_type,
+            args={i: v for i, v in enumerate(inputs)},  # pylint: disable=unnecessary-comprehension
+            continuation=create_computer(backend="tir"),
+            backend="tir",
+        )
+
+    # make targets
+    target = Target(target)
+    baseline_target = Target(baseline_target)
+    # fetch functions & prepare inputs
+    if isinstance(f_input_generator, str):
+        f_input_generator = get_global_func(f_input_generator)
+    if isinstance(f_check_metric, str):
+        f_check_metric = get_global_func(f_check_metric)
+    inputs = to_numpy(f_input_generator(original_mod))  # type: ignore
+    # build & run original result
+    original_res = to_numpy(build_and_run(original_mod, target=baseline_target, dev_type="cpu"))
+    scheduled_res = to_numpy(build_and_run(scheduled_mod, target=target, dev_type=dev_type))
+    # check metric
+    if f_check_metric(to_tvm_ndarray(original_res), to_tvm_ndarray(scheduled_res)):  # type: ignore
+        return True
+    else:
+        print(
+            ("\n\n").join(
+                [
+                    "Validation failed!",
+                    "Original Result:" + DELIMITOR + str(original_res),
+                    "Scheduled Result:" + DELIMITOR + str(scheduled_res),
+                    "Input:" + DELIMITOR + str(inputs),
+                    "Original IRModule:" + DELIMITOR + original_mod.script(),
+                    "Scheduled IRModule:" + DELIMITOR + scheduled_mod.script(),
+                ]
+            )
+        )
+        return False
+
+
+def main():
+    """Main function"""
+    describe()
+    database = ms.database.JSONDatabase(
+        path_workload=ARGS.path_workload, path_tuning_record=ARGS.path_tuning_record
+    )
+    assert Target(ARGS.target).kind.name in ["llvm", "cuda"]
+    dev_type = "cpu" if Target(ARGS.target).kind.name == "llvm" else "cuda"

Review Comment:
   ```suggestion
       if ARGS.target.kind.name == "llvm":
         dev_type = "cpu"
       elif ARGS.target.kind.name == "cuda":
         dev_type = "cuda"
       else:
         raise ValueError("Unsupported target kind: " + str(ARGS.target.kind.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] junrushao commented on a diff in pull request #12948: [MetaSchedule] Add JSON Database Validation Scripts

Posted by GitBox <gi...@apache.org>.
junrushao commented on code in PR #12948:
URL: https://github.com/apache/tvm/pull/12948#discussion_r1018267990


##########
python/tvm/meta_schedule/testing/validate_database.py:
##########
@@ -0,0 +1,283 @@
+# 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.
+"""JSON Database validation script"""
+from typing import Union, Callable, List
+from distutils.util import strtobool
+import argparse
+import logging
+import warnings
+import numpy as np  # type: ignore
+
+import tvm
+from tvm.target import Target
+from tvm.ir import IRModule
+from tvm.tir import Schedule
+from tvm import meta_schedule as ms
+from tvm.meta_schedule.testing.custom_builder_runner import run_module_via_rpc
+from tvm.meta_schedule.testing.tune_utils import create_computer, generate_input_data
+from tvm._ffi import get_global_func, register_func
+from tvm.support import describe
+
+DELIMITOR = "\n" + "-" * 30 + "\n"
+
+
+def _parse_args():
+    args = argparse.ArgumentParser()
+    args.add_argument(
+        "--path-workload",
+        type=str,
+        required=True,
+        help="The path to the database workload file.",
+    )
+    args.add_argument(
+        "--path-tuning-record",
+        type=str,
+        required=True,
+        help="The path to the database tuning record file.",
+    )
+    args.add_argument(
+        "--target",
+        type=str,
+        required=True,
+    )

Review Comment:
   ```suggestion
       args.add_argument(
           "--target",
           type=tvm.target.Target,
           required=True,
       )
   ```



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