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 2020/09/03 19:55:08 UTC

[GitHub] [incubator-tvm] trevor-m opened a new pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

trevor-m opened a new pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395


   This PR adds support for partitioning, compiling, and running the TensorRT BYOC target.
   
   # Building
   
   There are two new cmake flags:
   * `USE_TENSORRT=ON/OFF`: enables TensorRT code generation - this does not require TensorRT libraries
   * `USE_TENSORRT_GRAPH_RUNTIME=ON/OFF/"path/to/TensorRT": enables TensorRT runtime - this requires TensorRT libraries. A system wide install of TensorRT from a deb package or JetPack can be detected by "ON", but a .tar.gz installation requires you to provide path to the extracted TensorRT archive.
   
   # Usage
   
   The compilation target should be "cuda" to ensure that input and output args to the TensorRT functions are placed on the GPU.
   
   ```
   # Compilation
   from tvm.relay.op.contrib import tensorrt
   mod = tensorrt.partition_for_tensorrt(mod, params)
   with relay.build_config(opt_level=3):
     graph, lib, params = relay.build(mod, target="cuda", params=params)
   # Running inference is unchanged
   mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
   mod.run(...)
   ```
   
   # High level components
   
   ## Partitioning
   
   The annotation rules for TensorRT change depending on the version of TensorRT that is being targeted as well as the "batching mode". This can be configured with the `trt_version` and `use_implicit_batch` args of `partition_for_tensorrt`.
   
   If TVM was built against the TensorRT library, the linked version is used for partitioning instead.
   
   ## Codegen
   
   This implementation using the JSONRuntime `JSONSerializer` base class for codegen to serialize the relay expression to a json format.
   
   ## Runtime
   
   During runtime, the `TensorRTBuilder` class (`tensorrt_builder.cc`) is used to convert the json graph to a TensorRT `INetworkDefinition` using TensorRT APIs. It uses the op converter classes in `tensorrt_ops.cc` to do this. Then, the TensorRT engine is built, this process can take up to a few minutes because TensorRT will perform its optimizations at this point. The engine is cached for further inference calls.
   
   The runtime can be compiled against many TensorRT versions thanks to if guards I have added. It will work for TensorRT 5, 6, and 7.
   
   # Areas I'm looking for feedback and ideas
   
   1. TensorRT has parameters such as `max_workspace_size` and `use_implicit_batch` which I want the user to be able to supply in `partition_for_tensorrt`. These parameters need to be passed along to the codegen and stored in the serialized graph until runtime. `use_implicit_batch` also influences the partitioning rules. Currently, I'm using environment variables to pass these from python to the codegen in C++. I wonder if there is a better way to do this?
   
   2. I've implemented a transformation called `prune_tensorrt_subgraphs()` in `python/tvm/relay/op/contrib/tensorrt.py`. This is run after partitioning and allows me to decide whether to keep a subgraph or return it back to the typical TVM compilation path. This is needed because some subgraphs could be invalid - such as when the inputs have different batch sizes or for optimization purposes if the subgraph has no multiply-accumulates. I have also implemented a general version of this in C++, but it uses the global registry to allow each codegen target to define its own `is_invalid_subgraph` callback. In the future we can switch to the generic version if we find a better way to register the callbacks.
   
   3. The targeted tensorrt version needs to be accessed during annotation. I've put it in a global variable 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.

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Build and Deploy ResNet-18 with TensorRT
+----------------------------------------
+
+Create a Relay graph from a MXNet ResNet-18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be a cuda target.
+
+.. code:: python
+
+    target = "cuda"
+    with tvm.transform.PassContext(opt_level=3, config={'relay.ext.tensorrt.options': config}):
+        lib = relay.build(mod, target=target, params=params)
+
+
+Export the module.
+
+.. code:: python
+
+    lib.export_library('compiled.so')
+
+
+Load module and run inference. The first run will take longer because the TensorRT engine will have
+to be built.
+
+.. code:: python
+
+    ctx = tvm.gpu(0)
+    loaded_lib = tvm.runtime.load_module('compiled.so')
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib['default'](ctx))
+    input_data = np.random.uniform(0, 1, input_shape).astype(dtype)
+    gen_module.run(data=input_data)
+
+
+Partitioning and Compilation Settings
+----------------

Review comment:
       ```suggestion
   -------------------------------------
   ```

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Build and Deploy ResNet-18 with TensorRT
+----------------------------------------
+
+Create a Relay graph from a MXNet ResNet-18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be a cuda target.

Review comment:
       Better to briefly say something about the `config`. Like what it is and how users should use it.
   
   Update: I found a later section talking about this. Then we can just provide a pointer (e.g., we will introduce the `config` in details in [section link])

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Build and Deploy ResNet-18 with TensorRT
+----------------------------------------
+
+Create a Relay graph from a MXNet ResNet-18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be a cuda target.
+
+.. code:: python
+
+    target = "cuda"
+    with tvm.transform.PassContext(opt_level=3, config={'relay.ext.tensorrt.options': config}):
+        lib = relay.build(mod, target=target, params=params)
+
+
+Export the module.
+
+.. code:: python
+
+    lib.export_library('compiled.so')
+
+
+Load module and run inference. The first run will take longer because the TensorRT engine will have
+to be built.

Review comment:
       Better to emphasize "load module and run inference on the target machine, which must have USE_TENSORT_GRAPH_RUNTIME" enabled.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.

Review comment:
       It might be better to emphasize the scenario. For example, we can say these two flags also enables cross-compilation: USE_TENSORT=ON lets you build a module with TensorRT support on a host machine; while USE_TENSORT_GRAPH_RUNTIME=ON enables the TVM runtime on an edge device to execute the TensorRT module.




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

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



[GitHub] [incubator-tvm] leandron commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
leandron commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r483477103



##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.

Review comment:
       Maybe `ret` can be replaced by `mod` here ?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION

Review comment:
       minor: can be simplified to
   
   ```
   return linked_var len(linked_ver) == 3 else TENSORRT_VERSION
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \

Review comment:
       This comparison `args[0].checked_type.shape[0] == args[0].checked_type.shape[0]` needs some review as it is comparing something with itself. Maybe the right side is expected to be `args[1]` ?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) != 1 and int(attrs.axis) != 3:
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (5, 1, 5):
+        print("strided_slice: requires TensorRT version 5.1.5 or higher.")
+        return False
+    if args[0].checked_type.dtype != "float32":
+        print("strided_slice: only fp32 inputs are supported.")
+        return False

Review comment:
       This clause is repeated here, and can be removed maybe?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) != 1 and int(attrs.axis) != 3:
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:

Review comment:
       Suggestion: This could be simplified to `if input_rank not in (2, 3, 4):`.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) != 1 and int(attrs.axis) != 3:
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:

Review comment:
       Suggestion: This could be simplified to `if input_rank not in (2, 3, 4):`.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")

Review comment:
       I think it would be better to replace use of `print`, with the Python `logging` infrastructure.
   
   This applies for all uses of print in this class and other sources.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):

Review comment:
       Missing `max_worspace_size` on docstring

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))

Review comment:
       Is there any alternative to the use of environment variables o communicate these arguments from the partitioning to the codegen? If there is no alternative I think at least these environment variables should be documented somewhere.
   
   I don't know the right answer, so I'll just cc @mbaret and @comaniac 
   

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) != 1 and int(attrs.axis) != 3:

Review comment:
       Suggestion: This could be simplified to `if int(attrs.axis) not in (1, 3):`.




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

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



[GitHub] [incubator-tvm] areusch commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
areusch commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-701055475


   @trevor-m i agree this seems unrelated. can you try retriggering the CI just to double-check it's reproducible? there was an issue with flaky bundle_deploy tests a while back, but we thought we fixed that. what command are you running locally?


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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r500486187



##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any

Review comment:
       USE_TENSORRT?

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.

Review comment:
       Should we use USE_TENSORRT_GRAPH_RUNTIME or just USE_TENSORRT_RUNTIME? GRAPH_RUNTIME might be misleading because TVM has a graph runtime

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;

Review comment:
       no need to have `return`

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    DLOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    DLOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {
+    std::string cache_dir = dmlc::GetEnv("TVM_TENSORRT_CACHE_DIR", std::string(""));
+    if (cache_dir.empty()) return false;
+    std::string key = GetSubgraphKey();
+    std::string path = cache_dir + "/" + key + ".plan";
+    // Check if engine is in the cache.
+    std::ifstream infile(path, std::ios::binary);
+    if (!infile.good()) return false;
+    DLOG(INFO) << "Loading cached TensorRT engine from " << path;
+    infile.close();
+    std::string serialized_engine;
+    LoadBinaryFromFile(path, &serialized_engine);
+    // Deserialize engine
+    nvinfer1::IRuntime* runtime = nvinfer1::createInferRuntime(logger_);
+    TensorRTEngineAndContext engine_and_context;
+    engine_and_context.engine =
+        runtime->deserializeCudaEngine(&serialized_engine[0], serialized_engine.size(), nullptr);
+    engine_and_context.context = engine_and_context.engine->createExecutionContext();
+    // Load metadata
+    std::string meta_path = cache_dir + "/" + key + ".meta";
+    std::string serialized_meta;
+    LoadBinaryFromFile(meta_path, &serialized_meta);
+    std::istringstream is(serialized_meta);
+    dmlc::JSONReader reader(&is);
+    dmlc::JSONObjectReadHelper helper;
+    helper.DeclareField("inputs", &engine_and_context.inputs);
+    helper.DeclareField("outputs", &engine_and_context.outputs);
+    helper.ReadAllFields(&reader);
+    trt_engine_cache_[symbol_name_] = engine_and_context;
+    return true;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will save the engine to that
+   * directory so it can be loaded later.
+   */
+  void CacheEngineToDisk() {
+    std::string cache_dir = dmlc::GetEnv("TVM_TENSORRT_CACHE_DIR", std::string(""));
+    if (cache_dir.empty()) return;
+    std::string key = GetSubgraphKey();
+    std::string path = cache_dir + "/" + key + ".plan";
+    DLOG(INFO) << "Caching TensorRT engine to " << path;
+    // Serialize engine to disk
+    nvinfer1::IHostMemory* serialized_engine = trt_engine_cache_[symbol_name_].engine->serialize();
+    SaveBinaryToFile(path, std::string(static_cast<const char*>(serialized_engine->data()),
+                                       serialized_engine->size()));
+    serialized_engine->destroy();
+    // Serialize metadata
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    writer.BeginObject();
+    writer.WriteObjectKeyValue("inputs", trt_engine_cache_[symbol_name_].inputs);
+    writer.WriteObjectKeyValue("outputs", trt_engine_cache_[symbol_name_].outputs);
+    writer.EndObject();
+    std::string meta_path = cache_dir + "/" + key + ".meta";
+    SaveBinaryToFile(meta_path, os.str());
+  }
+
+  std::string GetSubgraphKey() {
+    // Using this key will only allow a single model per TVM_TENSORRT_CACHE_DIR directory. We could
+    // use instead use a hash of graph_json and all weights to allow many models in the same

Review comment:
       ```suggestion
       // instead use a hash of graph_json and all weights to allow many models in the same
   ```

##########
File path: src/runtime/contrib/tensorrt/tensorrt_utils.h
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/utils.h
+ * \brief Helper functions used by TensorRTBuilder or TrtOpConverters.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+
+#include <string>
+#include <vector>
+
+#include "NvInfer.h"
+
+// There is a conflict between cpplint and clang-format-10.
+// clang-format off
+#define TRT_VERSION_GE(major, minor, patch)                                                    \
+  ((NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
+  (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
+// clang-format on
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+/*!
+ * \brief Helper function to convert an vector to TRT Dims.
+ * \param vec Vector.
+ * \return TRT Dims.
+ */
+template <typename T>
+inline nvinfer1::Dims VectorToTrtDims(const std::vector<T>& vec) {
+  nvinfer1::Dims dims;
+  // Dims(nbDims=0, d[0]=1) is used to represent a scalar in TRT.
+  dims.d[0] = 1;
+  dims.nbDims = vec.size();
+  for (size_t i = 0; i < vec.size(); ++i) {
+    dims.d[i] = vec[i];
+  }
+  return dims;
+}
+
+/*!
+ * \brief Helper function to convert TRT Dims to vector.
+ * \param vec TRT Dims.
+ * \return Vector.
+ */
+inline std::vector<int> TrtDimsToVector(const nvinfer1::Dims& dims) {
+  return std::vector<int>(dims.d, dims.d + dims.nbDims);
+}
+
+/*!
+ * \brief Helper function to convert vector to string.
+ * \param vec Vector.
+ * \return Vector as a string.
+ */
+template <typename T>
+inline std::string DebugString(const std::vector<T>& vec) {

Review comment:
       maybe we have it, but it seems we don't use it in this PR?




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

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



[GitHub] [incubator-tvm] masahi edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
masahi edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686768783


   >     1. Currently, I'm using environment variables to pass these from python to the codegen in C++. I wonder if there is a better way to do this?
   
   How about using Config mechanism? I learned about this from ethos integration (thanks @mbaret) and it cleaned up my code as well. See the definition of ConfigNode below and its usage (grep for `GetConfig`).
   
   https://github.com/apache/incubator-tvm/blob/30cd2302e4078b3a8787e30d70fd79e5b729ec82/src/relay/backend/contrib/ethosn/codegen_ethosn.h#L219


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-696860539


   > Finished reviewing the C++ part and the tests. I am not familiar with the TRT APIs so I didn't review the op converter in details.
   > 
   > For tests, how long does it need to run all the tests? I'm afraid that running all unit tests here will slow the CI a lot.
   
   Thanks Cody, I measured the time on a AWS g4dn.4xlarge instance.
   
   Full with runtime: 6m2.021s
   Codegen only: 1m0.358s
   Codegen only, removed all integration test models except resnet18_v1 and mobilenet_v2: 0m51.536s
   
   Seems like the extra models in the integration tests do not consume much time.


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692316634


   > > Hmm, this seems like it would make the job of the `PruneTensorRTSubgraph` pass much more difficult. `PartitionGraph` already takes care of collecting the inputs and outputs of a subgraph and additional processing such as making sure there are no duplicate outputs. If `PruneTesnorRTCompilerRegion` was before `PartitionGraph`, it would have to duplicate a lot of that work. The idea of the pruning pass is that we should present each backend with the final subgraph exactly as it would be when it is passed to the codegen and the backend should decide if it is valid or not. Are you concerned about the overhead of partitioning a subgraph which would be later discarded?
   > > Btw just for referece, here is the general implementation of PruneSubgraph that I originally implemented: [trevor-m@06015a4](https://github.com/trevor-m/tvm/commit/06015a4617cfaad56adcaa0c71b485d6bd711128)
   > 
   > My main concern was that it would be tedious to have a `partition_graph -> revert_some_partitions` flow. Also in this case, your post-processing pass depends on the partition pass and may fail along with the change of the partition pass. If this requirement is important, I'd even prefer to add post-processing feature to the partition pass that allows you to provide a packed function to check if a partitioned function is valid.
   > 
   > On the other hand, in order to not block this PR for too long, we can maybe follow the current flow first, and discuss a plan of refactoring the partition pass to better support this requirement.
   > 
   > @zhiics do you have any suggestion?
   
   Thanks! That makes sense. My implementation seems tightly coupled to how PartitionGraph works.
   
   I like the idea of adding the callback to PartitionGraph. After it puts together the function, it can check if there is a validation function registered and call it to see if it should keep the subgraph or not. Both MXNet and TF have a mechanism like this as a final check on the subgraph in their partitioning algorithms.
   
   I agree that solving this problem is probably best done in a separate PR.


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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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






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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r500554166



##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.

Review comment:
       USE_TENSORRT_GRAPH_RUNTIME is referring to TVM's graph runtime. However since this could be also be used with VM `USE_TENSORRT_RUNTIME` is perhaps a better 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.

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   @t-vi looks like this is a common issue in cpplint as it only uses regular expression to lint the code. While it's not straightforward to fix cpplint, you may work around this issue by rewriting the code like
   ```c
   #define TRT_VERSION_GE(major, minor, patch) (                                                 \
     (NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
     (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
   ```
   
   So that both cpplint and clang-format are happy.


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-712488241


   Thanks @trevor-m @comaniac @lhutton1 @leandron 


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r492835284



##########
File path: src/runtime/contrib/tensorrt/tensorrt_ops.h
##########
@@ -0,0 +1,208 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_ops.h
+ * \brief Converters from Relay ops into TensorRT layers. Converters should
+ * inherit from TrtOpConverter and implement the Convert() method.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+
+#include <algorithm>
+#include <cmath>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_utils.h"
+
+#if TRT_VERSION_GE(6, 0, 1)
+#define TRT_HAS_IMPLICIT_BATCH(params) (params->network->hasImplicitBatchDimension())
+#else
+#define TRT_HAS_IMPLICIT_BATCH(params) (true)
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+
+/*!
+ * \brief An input to a op may be either kTensor in the case of nvinfer::ITensor*
+ * or kWeight for nvinfer1::Weights.
+ */
+enum TrtInputType {
+  kTensor,
+  kWeight,
+};
+
+/*!
+ * \brief An input to a TrtOpConverter. The type of the input is either kTensor
+ * or kWeight. For kTensor, "tensor" contains the input tensor. For kWeight,
+ * "weight" contains the input weight and "weight_shape" contains the shape.
+ */
+struct TrtOpInput {
+  /*! \brief If type is kTensor, will store input tensor. */
+  nvinfer1::ITensor* tensor;
+
+  /*! \brief If type is kWeight, will store input weight. */
+  nvinfer1::Weights weight;
+
+  /*! \brief Whether the input is in tensor or weight. */
+  TrtInputType type;
+
+  /*! \brief If type is kWeight, will store weight shape. */
+  std::vector<int> weight_shape;
+
+  explicit TrtOpInput(nvinfer1::ITensor* tensor)
+      : tensor(tensor), weight({nvinfer1::DataType::kFLOAT, nullptr, 0}), type(kTensor) {}
+  TrtOpInput(nvinfer1::Weights weight, const std::vector<int>& shape)
+      : tensor(nullptr), weight(weight), type(kWeight), weight_shape(shape) {}
+};
+
+/*! \brief Parameters to convert an Op from relay to TensorRT. */
+struct AddTrtLayerParams {

Review comment:
       Renamed to `TensorRTOpConverterParams`




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

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



[GitHub] [incubator-tvm] lhutton1 commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
lhutton1 commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692850735


   Np @trevor-m, it seems that way, at least that's where I added the Arm Compute Library doc


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       The message is a bit confusing. `USE_TENSORRT` means enabling the TensorRT codegen for graph partitininog. It doesn't require TensorRT to be available in the system environment. IIUC, maybe it's better to say "Build with TensorRT codegen", although I just found that "Build with Arm Compute Library" has the same issue.
   
   @lhutton1 could you also share your thoughts about this?




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   Given that we are not enabling TRT codegen on CI now due to the lack of TensorRT, I suggest we bypass this issue first to get the PR merged. Meanwhile, it would be better to have a troubleshooting in the TRT codegen tutorial.


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692843402


   > Just a couple of minor suggestions and comments - looks good overall, without regard to comments/concerns above.
   > 
   > One overall suggestion from a users perspective, it might be useful to write a beginners guide stating how to install and build with TensorRT support. Feel free to ignore though.
   
   Thanks @lhutton1 for the review! Is `docs/deploy` the typical place for a guide like that?


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

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



[GitHub] [incubator-tvm] lhutton1 commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
lhutton1 commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r500863217



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       Agreed this is confusing. I think changing to `..._CODEGEN` would be a better description of what the option actually does.




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   Looks like a conflict between cpplint and clang-format-10. clang-format-10 result seems more reasonable, so we may need to fix the cpplint.
   
   cc @areusch 


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r489717587



##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):
+        """
+        Visit an expression while replacing vars according to var_map. Used by
+        SubgraphRemover/PruneSubgraphs to return a subgraph originally partitioned to TRT back to TVM.
+        """
+        def __init__(self, var_map):
+            ExprMutator.__init__(self)
+            self.var_map = var_map
+
+        def visit_var(self, var):
+            if var in self.var_map:
+                return self.var_map[var]
+            return super().visit_var(var)
+
+    class SubgraphRemover(ExprMutator):

Review comment:
       I originally tried that approach. However, when the tensorrt subgraphs are inlined, TVM will try to optimize the code in the tensorrt subgraphs (for example it will change conv2d to contrib_conv2d_winograd_without_weight_transform). I don't want this.




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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r500619581



##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;

Review comment:
       We only need to read the values once, so returning allows us to avoid doing the same thing for all of the nodes




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

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



[GitHub] [incubator-tvm] lhutton1 commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
lhutton1 commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r490817376



##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+-------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be "cuda".
+
+.. code:: python
+
+    target = "cuda"
+    with tvm.transform.PassContext(opt_level=3, config={'relay.ext.tensorrt.options': config}):
+        lib = relay.build(mod, target=target, params=params)
+
+
+Export the module.
+
+.. code:: python
+
+    lib.export_library('compiled.so')
+
+
+Run inference.
+
+.. code:: python
+
+    ctx = tvm.gpu(0)
+    loaded_lib = tvm.runtime.load_module('compiled.so')
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib['default'](ctx))
+    input_data = np.random.uniform(0, 1, input_shape).astype(dtype)
+    gen_module.run(data=input_data)
+
+
+Operator support
+----------------
++------------------------+------------------------------------+
+|       Relay Node       |              Remarks               |
++========================+====================================+
+| nn.relu                |                                    |
++------------------------+------------------------------------+
+| sigmoid                |                                    |
++------------------------+------------------------------------+
+| tanh                   |                                    |
++------------------------+------------------------------------+
+| nn.batch_norm          |                                    |
++------------------------+------------------------------------+
+| nn.softmax             |                                    |
++------------------------+------------------------------------+
+| nn.conv2d              |                                    |
++------------------------+------------------------------------+
+| nn.dense               |                                    |
++------------------------+------------------------------------+
+| nn.bias_add            |                                    |
++------------------------+------------------------------------+
+| add                    |                                    |
++------------------------+------------------------------------+
+| subtract               |                                    |
++------------------------+------------------------------------+
+| multiply               |                                    |
++------------------------+------------------------------------+
+| divide                 |                                    |
++------------------------+------------------------------------+
+| power                  |                                    |
++------------------------+------------------------------------+
+| maximum                |                                    |
++------------------------+------------------------------------+
+| minimum                |                                    |
++------------------------+------------------------------------+
+| nn.max_pool2d          |                                    |
++------------------------+------------------------------------+
+| nn.avg_pool2d          |                                    |
++------------------------+------------------------------------+
+| nn.global_max_pool2d   |                                    |
++------------------------+------------------------------------+
+| nn.global_avg_pool2d   |                                    |
++------------------------+------------------------------------+
+| exp                    |                                    |
++------------------------+------------------------------------+
+| log                    |                                    |
++------------------------+------------------------------------+
+| sqrt                   |                                    |
++------------------------+------------------------------------+
+| abs                    |                                    |
++------------------------+------------------------------------+
+| negative               |                                    |
++------------------------+------------------------------------+
+| nn.batch_flatten       |                                    |
++------------------------+------------------------------------+
+| expand_dims            |                                    |
++------------------------+------------------------------------+
+| squeeze                |                                    |
++------------------------+------------------------------------+
+| concatenate            |                                    |
++------------------------+------------------------------------+
+| nn.conv2d_transpose    |                                    |
++------------------------+------------------------------------+
+| transpose              |                                    |
++------------------------+------------------------------------+
+| layout_transform       |                                    |
++------------------------+------------------------------------+
+| reshape                |                                    |
++------------------------+------------------------------------+
+| nn.pad                 |                                    |
++------------------------+------------------------------------+
+| sum                    |                                    |
++------------------------+------------------------------------+
+| prod                   |                                    |
++------------------------+------------------------------------+
+| max                    |                                    |
++------------------------+------------------------------------+
+| min                    |                                    |
++------------------------+------------------------------------+
+| mean                   |                                    |
++------------------------+------------------------------------+
+| nn.adaptive_max_pool2d |                                    |
++------------------------+------------------------------------+
+| nn.adaptive_avg_pool2d |                                    |
++------------------------+------------------------------------+
+| clip                   | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| nn.leaky_relu          | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| sin                    | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| cos                    | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| atan                   | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| ceil                   | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| floor                  | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| strided_slice          | Requires TensorRT 5.1.5 or greater |
++------------------------+------------------------------------+
+| nn.conv3d              | Requires TensorRT 6.0.1 or greater |
++------------------------+------------------------------------+
+| nn.max_pool3d          | Requires TensorRT 6.0.1 or greater |
++------------------------+------------------------------------+
+| nn.avg_pool3d          | Requires TensorRT 6.0.1 or greater |
++------------------------+------------------------------------+
+| nn.conv3d_transpose    | Requires TensorRT 6.0.1 or greater |
++------------------------+------------------------------------+
+
+
+Runtime Settings
+----------------
+
+There are some additional options which can be configured at runtime using environment variables.
+
+* Automatic FP16 Conversion - Environment variable ``TVM_TENSORRT_USE_FP16=1`` can be set to
+  automatically convert the TensorRT components of your model to 16-bit floating point precision.
+  This can greatly increase performance, but may cause some slight loss in the model accuracy.
+* During the first inference, the runtime will invoke the TensorRT API to build an engine. This can
+  be time consuming, so you can set ``TVM_TENSORRT_CACHE_DIR`` to point to a directory to save
+  these built engines to on the disk. The next time you load the model and give it the same
+  directory, the runtime will load the already built engines to avoid the long warmup time. A
+  unique directory is required for each model.
+* TensorRT has a paramter to configure the maximum amount of scratch space that each layer in the
+  model can use. It is generally best to use the highest value which does not cause you to run out
+  of memory. You can use ``TVM_TENSORRT_MAX_WORKSPACE_SIZE`` to override this by specifying the
+  workspace size in bytes you would like to use.
+
+Adding a new operator
+---------------------
+To add support for a new operator, there are a series of files we need to make changes to:
+
+* `python/relay/op/contrib/tensorrt.py` This file contains the annotation rules for TensorRT. These
+  determine which operators and their attributes that are supported.
+* `src/runtime/contrib/arm_compute_lib/tensorrt_ops.cc` Create a new op converter class which

Review comment:
       ```suggestion
   * `src/runtime/contrib/tensorrt/tensorrt_ops.cc` Create a new op converter class which
   ```

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       Could you override the default SaveToBinary in the json runtime and optionally save the engine if one exists (and/or based on a config option)? When LoadFromBinary is called, since you have defined your own serialization method you can check for the existence of the engine and load it back. Essentially you have two different serialization/deserialization methods which you can alternate between in LoadFrom/SaveToBinary




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   Zhi just pointed to me offline about how to access pass context configs in Python. Here is an example:
   
   ```python
   import tvm
   with tvm.transform.PassContext(config={"relay.fallback_device_type": 5}):
       pass_ctx = tvm.transform.PassContext.current()
       print(pass_ctx.config["relay.fallback_device_type"])
   ```


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692298168


   > For the rest 2 points.
   > 
   >     2. Is that possible to move the pass before partitioning but after merge compiler region (like `PruneTesnorRTCompilerRegion`)? After the merge compiler region pass you should get the Relay graph with almost the same semantic as partitioning. If you could have a pass checking each compiler region for your constraints, you can probably just remove the region you don't want, so that you should get only valid partitioned functions.
   > 
   >     3. Can the TensorRT version be obtained via an API call in C++? Something like `tensorrt::get_version()`? If so you can register a global symbol and pass the version to Python so that it can be used by the annotator.
   > 
   > 
   > ```python
   > def conv2d(...):
   >     if not tvm.get_global_func("relay.tensorrt.version", True):
   >         return False
   >     ver = tvm.get_global_func("relay.tensorrt.version")
   >     if ver == '1.0':
   >         return True
   >     return False
   > ```
   > 
   > If you need manually set up the TensorRT version, then it could be like this: Let user specify it in `config.cmake` and we pass the value to a macro in C++ so that you could simply return the value. The drawback of this solution is that it needs to rebuild TVM to annotate different TensorRT versions, and I'm not sure if that makes sense to you.
   
   Thanks @comaniac!
   
   > 2. Is that possible to move the pass before partitioning but after merge compiler region (like `PruneTesnorRTCompilerRegion`)? After the merge compiler region pass you should get the Relay graph with almost the same semantic as partitioning. If you could have a pass checking each compiler region for your constraints, you can probably just remove the region you don't want, so that you should get only valid partitioned functions.
   
   Hmm, this seems like it would make the job of the `PruneTensorRTSubgraph` pass much more difficult. `PartitionGraph` already takes care of collecting the inputs and outputs of a subgraph and additional processing such as making sure there are no duplicate outputs. If `PruneTesnorRTCompilerRegion` was before `PartitionGraph`, it would have to duplicate a lot of that work. The idea of the pruning pass is that we should present each backend with the final subgraph exactly as it would be when it is passed to the codegen and the backend should decide if it is valid or not. Are you concerned about the overhead of partitioning a subgraph which would be later discarded?
   
   Btw just for referece, here is the general implementation of PruneSubgraph that I originally implemented: https://github.com/trevor-m/tvm/commit/06015a4617cfaad56adcaa0c71b485d6bd711128
   
   > 3. Can the TensorRT version be obtained via an API call in C++? Something like `tensorrt::get_version()`? If so you can register a global symbol and pass the version to Python so that it can be used by the annotator.  If you need manually set up the TensorRT version, then it could be like this: Let user specify it in `config.cmake` and we pass the value to a macro in C++ so that you could simply return the value. The drawback of this solution is that it needs to rebuild TVM to annotate different TensorRT versions, and I'm not sure if that makes sense to you.
   
   I have already created an API to retrieve the TRT version if TVM is compiled with the TRT runtime enabled. However, one of our use cases is to use TVM on a CPU-only instance to cross-compile models. For that use case, we want to be able to target compilation for different TRT versions - this affects the partitioning rules mostly. I don't think having to rebuild TVM for each target version will be a good solution.
   
   Is it possible for my annotation functions to access the pass context and therefore a TRT config that I will be adding as @masahi suggested? I don't see any other python code accessing the PassContext though...
   


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r492407348



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       Thanks for the review Cody!
   
   You're right, the names aren't really that clear here. Originally, I had them as `USE_TENSORRT_CODEGEN` for codegen only and `USE_TENSORRT` for both codegen and runtime. I changed them to match the ACL definitions.




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

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



[GitHub] [incubator-tvm] masahi edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
masahi edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686768783


   >     1. Currently, I'm using environment variables to pass these from python to the codegen in C++. I wonder if there is a better way to do this?
   
   How about using Config mechanism? I learned about this from ethos integration (thanks @mbaret) and it cleaned up my code as well. See the definition of ConfigNode below and its usage.
   
   https://github.com/apache/incubator-tvm/blob/30cd2302e4078b3a8787e30d70fd79e5b729ec82/src/relay/backend/contrib/ethosn/codegen_ethosn.h#L219


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-704464026


   @lhutton1 @leandron please take another look per https://tvm.apache.org/docs/contribute/code_review.html#approve-and-request-changes-explicitly


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

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



[GitHub] [incubator-tvm] leandron commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
leandron commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r483475694



##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION

Review comment:
       minor: can be simplified to
   
   ```
   return linked_var if len(linked_ver) == 3 else TENSORRT_VERSION
   ```




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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707258309


   Hey @areusch 
   I found that the failures for `Test MISRA-C runtime` are related to my new CMake option `USE_TENSORRT_CODEGEN`. When `USE_TENSORRT_CODEGEN` is `ON`, the CPP tests start to segfault. Can you see anything in my cmake rules that looks like it could be causing the issue?


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686770757


   > > ```
   > > 1. Currently, I'm using environment variables to pass these from python to the codegen in C++. I wonder if there is a better way to do this?
   > > ```
   > 
   > How about using Config mechanism? I learned about this from ethos integration (thanks @mbaret) and it cleaned up my code as well. See the definition of ConfigNode below and its usage (grep for `GetConfig`).
   > 
   > https://github.com/apache/incubator-tvm/blob/30cd2302e4078b3a8787e30d70fd79e5b729ec82/src/relay/backend/contrib/ethosn/codegen_ethosn.h#L219
   
   Thanks @masahi! Let me look into this.


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707390645


   I agree that we can merge it first. But before that, @trevor-m could you rebase against the master and run the tests again locally to see if all of them pass? I am not sure if everything is oaky after the diagnostic error reporting was merged.


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

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



[GitHub] [incubator-tvm] leandron commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
leandron commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r483526749



##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))

Review comment:
       I see this is covered on your PR original message :) - so you can ignore this one.




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

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



[GitHub] [incubator-tvm] trevor-m edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692298168


   Thanks @comaniac!
   
   > 2. Is that possible to move the pass before partitioning but after merge compiler region (like `PruneTesnorRTCompilerRegion`)? After the merge compiler region pass you should get the Relay graph with almost the same semantic as partitioning. If you could have a pass checking each compiler region for your constraints, you can probably just remove the region you don't want, so that you should get only valid partitioned functions.
   
   Hmm, this seems like it would make the job of the `PruneTensorRTSubgraph` pass much more difficult. `PartitionGraph` already takes care of collecting the inputs and outputs of a subgraph and additional processing such as making sure there are no duplicate outputs. If `PruneTesnorRTCompilerRegion` was before `PartitionGraph`, it would have to duplicate a lot of that work. The idea of the pruning pass is that we should present each backend with the final subgraph exactly as it would be when it is passed to the codegen and the backend should decide if it is valid or not. Are you concerned about the overhead of partitioning a subgraph which would be later discarded?
   
   Btw just for referece, here is the general implementation of PruneSubgraph that I originally implemented: https://github.com/trevor-m/tvm/commit/06015a4617cfaad56adcaa0c71b485d6bd711128
   
   > 3. Can the TensorRT version be obtained via an API call in C++? Something like `tensorrt::get_version()`? If so you can register a global symbol and pass the version to Python so that it can be used by the annotator.  If you need manually set up the TensorRT version, then it could be like this: Let user specify it in `config.cmake` and we pass the value to a macro in C++ so that you could simply return the value. The drawback of this solution is that it needs to rebuild TVM to annotate different TensorRT versions, and I'm not sure if that makes sense to you.
   
   I have already created an API to retrieve the TRT version if TVM is compiled with the TRT runtime enabled. However, one of our use cases is to use TVM on a CPU-only instance to cross-compile models. For that use case, we want to be able to target compilation for different TRT versions - this affects the partitioning rules mostly. I don't think having to rebuild TVM for each target version will be a good solution.
   
   Is it possible for my annotation functions to access the pass context and therefore a TRT config that I will be adding as @masahi suggested? I don't see any other python code accessing the PassContext though...
   


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707922105


   I think we can enable the test and merge after #6679 is landed since its pretty close already?


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r492832239



##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {

Review comment:
       There can also be weights when `GetOpType() == "const"`




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: src/relay/backend/contrib/tensorrt/codegen.cc
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/tensorrt/codegen.cc
+ * \brief Implementation of the TensorRT JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../utils.h"
+#include "../codegen_json/codegen_json.h"
+
+#if TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#endif
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*! \brief Attributes to store the compiler options for TensorRT. */
+struct TensorRTCompilerConfigNode : public tvm::AttrsNode<TensorRTCompilerConfigNode> {
+  Array<Integer> tensorrt_version;
+  bool use_implicit_batch;
+  size_t max_workspace_size;
+  bool remove_no_mac_subgraphs;
+
+  TVM_DECLARE_ATTRS(TensorRTCompilerConfigNode, "ext.attrs.TensorRTCompilerConfigNode") {
+    TVM_ATTR_FIELD(tensorrt_version)
+        .describe("TensorRT version as (major, minor, patch).")
+        .set_default(Array<Integer>({6, 0, 1}));
+    TVM_ATTR_FIELD(use_implicit_batch).set_default(true);
+    TVM_ATTR_FIELD(max_workspace_size).set_default(size_t(1) << 30);
+    TVM_ATTR_FIELD(remove_no_mac_subgraphs).set_default(false);
+  }
+};
+
+class TensorRTCompilerConfig : public Attrs {
+ public:
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TensorRTCompilerConfig, Attrs,
+                                            TensorRTCompilerConfigNode);
+};
+
+TVM_REGISTER_NODE_TYPE(TensorRTCompilerConfigNode);
+TVM_REGISTER_PASS_CONFIG_OPTION("relay.ext.tensorrt.options", TensorRTCompilerConfig);
+
+/*!
+ * \brief Generates an TensorRTModule from a relay expression by serializing the expression to a
+ * json representation. TensorRT is not required here because use of TensorRT APIs is deferred until
+ * runtime.
+ */
+class TensorRTJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  TensorRTJSONSerializer(const std::string& symbol, const Expr& expr)
+      : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+
+    std::vector<JSONGraphNodeEntry> inputs;
+    for (const auto& arg : cn->args) {
+      auto res = VisitExpr(arg);
+      inputs.insert(inputs.end(), res.begin(), res.end());
+    }
+    auto node = std::make_shared<JSONGraphNode>(name,     /* name_ */
+                                                "kernel", /* op_type_ */
+                                                inputs, 1 /* num_outputs_ */);
+    if (name == "nn.pad") {
+      SetPadNodeAttribute(node, cn);
+    } else if (name == "strided_slice") {
+      SetStridedSliceNodeAttribute(node, cn);
+    } else {
+      SetCallNodeAttribute(node, cn);
+    }
+    // These attributes are global to the whole module.
+    SaveGlobalAttributes(node);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  void SetPadNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* pad_attr = cn->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    const int dim_h = (p.size() == 5) ? 3 : 2;
+    const int dim_w = (p.size() == 5) ? 4 : 3;
+    std::vector<std::string> padding = {std::to_string(p[dim_h][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_h][1].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    node->SetAttr("padding", padding_attr);
+  }
+
+  void SetStridedSliceNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* attrs = cn->attrs.as<StridedSliceAttrs>();
+    CHECK(attrs);
+    CHECK(attrs->begin && attrs->end && attrs->strides);
+    const bool default_strides =
+        !attrs->strides.value().defined() || attrs->strides.value().size() == 0;
+    auto ishape = backend::GetShape(cn->args[0]->checked_type());
+
+    auto process_slice_index = [](Integer x, int default_value, int dim_value) {
+      if (!x.defined()) return default_value;
+      int value = x.as<IntImmNode>()->value;
+      if (value < 0) value += dim_value;
+      return value;
+    };
+
+    std::vector<std::string> start, size, strides;
+    for (size_t i = 0; i < attrs->begin.value().size(); ++i) {
+      const int begin_value = process_slice_index(attrs->begin.value()[i], 0, ishape[i]);
+      const int end_value = process_slice_index(attrs->end.value()[i], ishape[i], ishape[i]);
+      const int stride_value = (default_strides || i >= attrs->strides.value().size() ||
+                                !attrs->strides.value()[i].defined())
+                                   ? 1
+                                   : attrs->strides.value()[i].as<IntImmNode>()->value;
+      CHECK_GT(stride_value, 0);
+      const int size_value = (end_value - begin_value + stride_value - 1) / stride_value;
+      CHECK_GE(begin_value, 0);
+      CHECK_GT(size_value, 0);
+      start.push_back(std::to_string(begin_value));
+      size.push_back(std::to_string(size_value));
+      strides.push_back(std::to_string(stride_value));
+    }
+    std::vector<dmlc::any> start_attr, size_attr, strides_attr;
+    start_attr.emplace_back(start);
+    size_attr.emplace_back(size);
+    strides_attr.emplace_back(strides);
+    node->SetAttr("start", start_attr);
+    node->SetAttr("size", size_attr);
+    node->SetAttr("strides", strides_attr);
+  }
+
+  void SaveGlobalAttributes(std::shared_ptr<JSONGraphNode> node) {
+    auto ctx = transform::PassContext::Current();
+    auto cfg = ctx->GetConfig<TensorRTCompilerConfig>("relay.ext.tensorrt.options");
+    if (!cfg.defined()) {
+      cfg = AttrsWithDefaultValues<TensorRTCompilerConfig>();
+    }
+    CHECK_EQ(cfg.value()->tensorrt_version.size(), 3);
+    std::vector<std::string> tensorrt_version = {std::to_string(cfg.value()->tensorrt_version[0]),
+                                                 std::to_string(cfg.value()->tensorrt_version[1]),
+                                                 std::to_string(cfg.value()->tensorrt_version[2])};
+    std::vector<std::string> use_implicit_batch = {std::to_string(cfg.value()->use_implicit_batch)};
+    std::vector<std::string> max_workspace_size = {std::to_string(cfg.value()->max_workspace_size)};
+    std::vector<dmlc::any> tensorrt_version_attr, use_implicit_batch_attr, max_workspace_size_attr;
+    tensorrt_version_attr.emplace_back(tensorrt_version);
+    use_implicit_batch_attr.emplace_back(use_implicit_batch);
+    max_workspace_size_attr.emplace_back(max_workspace_size);
+    node->SetAttr("tensorrt_version", tensorrt_version_attr);
+    node->SetAttr("use_implicit_batch", use_implicit_batch_attr);
+    node->SetAttr("max_workspace_size", max_workspace_size_attr);
+  }
+};
+
+/*!
+ * \brief Create a runtime module for TensorRT.
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module TensorRTCompiler(const ObjectRef& ref) {
+  CHECK(ref->IsInstance<FunctionNode>()) << "The input ref is expected to be a Relay function.";
+  Function func = Downcast<Function>(ref);
+  std::string func_name = backend::GetExtSymbol(func);
+
+  TensorRTJSONSerializer serializer(func_name, func);
+  serializer.serialize();
+  std::string graph_json = serializer.GetJSON();
+  auto param_names = serializer.GetParams();
+  const auto* pf = runtime::Registry::Get("runtime.tensorrt_runtime_create");
+  CHECK(pf != nullptr) << "Cannot find JSON runtime module to create";

Review comment:
       Improve the error message to be more TensorRT specific.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_builder.h
##########
@@ -0,0 +1,159 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_builder.h
+ * \brief The TensorRTBuilder class can be used to convert a JSONRuntime graph into a TRT engine
+ * which can be used for inference.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_logger.h"
+#include "tensorrt_ops.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+/*!
+ * \brief The product of TensorRTBuilder which provides everything needed to
+ * perform inference.
+ */
+struct TrtEngineAndContext {

Review comment:
       Better to be consistent with `TensorRTBuilder`: `TensorRTEngineAndContext`.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_ops.h
##########
@@ -0,0 +1,208 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_ops.h
+ * \brief Converters from Relay ops into TensorRT layers. Converters should
+ * inherit from TrtOpConverter and implement the Convert() method.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+
+#include <algorithm>
+#include <cmath>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_utils.h"
+
+#if TRT_VERSION_GE(6, 0, 1)
+#define TRT_HAS_IMPLICIT_BATCH(params) (params->network->hasImplicitBatchDimension())
+#else
+#define TRT_HAS_IMPLICIT_BATCH(params) (true)
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+
+/*!
+ * \brief An input to a op may be either kTensor in the case of nvinfer::ITensor*
+ * or kWeight for nvinfer1::Weights.
+ */
+enum TrtInputType {
+  kTensor,
+  kWeight,
+};
+
+/*!
+ * \brief An input to a TrtOpConverter. The type of the input is either kTensor
+ * or kWeight. For kTensor, "tensor" contains the input tensor. For kWeight,
+ * "weight" contains the input weight and "weight_shape" contains the shape.
+ */
+struct TrtOpInput {
+  /*! \brief If type is kTensor, will store input tensor. */
+  nvinfer1::ITensor* tensor;
+
+  /*! \brief If type is kWeight, will store input weight. */
+  nvinfer1::Weights weight;
+
+  /*! \brief Whether the input is in tensor or weight. */
+  TrtInputType type;
+
+  /*! \brief If type is kWeight, will store weight shape. */
+  std::vector<int> weight_shape;
+
+  explicit TrtOpInput(nvinfer1::ITensor* tensor)
+      : tensor(tensor), weight({nvinfer1::DataType::kFLOAT, nullptr, 0}), type(kTensor) {}
+  TrtOpInput(nvinfer1::Weights weight, const std::vector<int>& shape)
+      : tensor(nullptr), weight(weight), type(kWeight), weight_shape(shape) {}
+};
+
+/*! \brief Parameters to convert an Op from relay to TensorRT. */
+struct AddTrtLayerParams {

Review comment:
       What does "Add" means here?

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):
+    if skip_codegen_test():
+        return
+    f, input_shapes, is_param = config
+    params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+    input_dict = {
+        k: np.random.uniform(-1, 1, v).astype(np.float32)
+        for k, v in input_shapes.items()
+        if k not in is_param
+    }
+
+    # Run TRT
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    # Run reference
+    mod = tvm.IRModule()
+    mod["main"] = f
+    with tvm.transform.PassContext(opt_level=3):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    assert len(results) == len(ref_results)
+    for i in range(len(results)):
+        res = results[i].asnumpy()
+        ref_res = ref_results[i].asnumpy()
+        assert res.shape == ref_res.shape
+        tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+
+def run_and_verify_model(model):
+    if skip_codegen_test():
+        return
+
+    def compile_and_run(i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+
+        def check_trt_used(graph):
+            import json
+
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum(
+                [
+                    1
+                    for n in graph["nodes"]
+                    if n.get("attrs", {}).get("func_name", "").startswith("tensorrt_")
+                ]
+            )
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={"data": input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod, config = tensorrt.partition_for_tensorrt(mod, params)
+            with tvm.transform.PassContext(
+                opt_level=3, config={"relay.ext.tensorrt.options": config}
+            ):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+            check_trt_used(graph)
+        else:
+            with tvm.transform.PassContext(opt_level=3):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+
+        if skip_runtime_test():
+            return
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return res
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    res = compile_and_run(i_data, input_shape, dtype, use_trt=True)
+    ref_res = compile_and_run(i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+    tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)
+
+
+def test_tensorrt_simple():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3, 1, 1)
+    zshape = (1, 1, 1, 1)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.var("y", shape=(yshape), dtype=dtype)
+    z = relay.var("z", shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_not_compatible():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 32, 14, 14)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_serialize():
+    if skip_codegen_test():
+        return
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    block = get_model("resnet18_v1", pretrained=True)
+    mod, params = relay.frontend.from_mxnet(
+        block, shape={"data": (1, 3, 224, 224)}, dtype="float32"
+    )
+    # Compile
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        lib = relay.build(mod, "cuda", params=params)
+    # Serialize
+    lib.export_library("compiled.so")
+    # Deserialize
+    loaded_lib = tvm.runtime.load_module("compiled.so")
+    # Run
+    if skip_runtime_test():
+        return
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib["default"](tvm.gpu(0)))
+    i_data = np.random.uniform(0, 1, (1, 3, 224, 224)).astype("float32")
+    for i in range(10):
+        gen_module.run(data=i_data)
+
+
+def test_conv2d():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(16, 32, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+        dilation=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    for k_shape, groups in [((16, 32, 3, 3), 1), ((32, 1, 3, 3), 32)]:
+        for padding in [(0, 0), (1, 1)]:
+            for strides in [(1, 1), (2, 2)]:
+                for dilation in [(1, 1), (2, 2)]:
+                    run_and_verify(
+                        get_graph(
+                            k_shape=k_shape,
+                            groups=groups,
+                            padding=padding,
+                            strides=strides,
+                            dilation=dilation,
+                        )
+                    )
+
+
+def test_conv2d_nhwc():
+    def get_graph(x_shape=(1, 8, 8, 32), k_shape=(3, 3, 32, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=16,
+            kernel_size=(3, 3),
+            data_layout="NHWC",
+            kernel_layout="HWIO",
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_conv2d_weights_const():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(16, 32, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+        dilation=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.const(np.ones(k_shape).astype("float32"))
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_conv2d_weights_transposed():
+    def get_graph(x_shape=(1, 32, 9, 9), k_shape=(3, 3, 32, 16), order=(3, 2, 0, 1)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        kernel_t = relay.transpose(kernel, order)
+        # Conv2d requires constant weights in TensorRT, so the weights should be transposed by
+        # FoldConstant.
+        out = relay.nn.conv2d(x, kernel_t, channels=k_shape[order[0]], kernel_size=(3, 3))
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_dense():
+    def get_graph(x_shape=(1, 16), k_shape=(32, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        # Dense requires constant weights in TensorRT, so the weights are transposed by us.
+        out = relay.nn.dense(x, kernel, units=k_shape[0])
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_bias_add():
+    def get_graph(x_shape=(1, 16), channels=16):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        bias = relay.var("bias", shape=(channels,), dtype="float32")
+        out = relay.nn.bias_add(x, bias)
+        f = relay.Function([x, bias], out)
+        return f, {"x": x_shape, "bias": (channels,)}, ["bias"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph((1, 6, 3, 4), 6))
+
+
+def test_pool2d():
+    def get_graph(
+        op,
+        x_shape=(1, 3, 32, 32),
+        pool_size=(2, 2),
+        strides=(2, 2),
+        padding=(0, 0),
+        ceil_mode=False,
+        count_include_pad=None,
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if count_include_pad is not None:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+                count_include_pad=count_include_pad,
+            )
+        else:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for pool_size in [(2, 2), (3, 3)]:
+        for strides in [(1, 1), (2, 2)]:
+            for padding in [(0, 0), (1, 1), (0, 0, 1, 1)]:
+                for ceil_mode in [False, True]:
+                    # Skip "the padding size is larger than or equal to the filter size for exclusive-counting pooling"
+                    if pool_size == (2, 2) and padding == (0, 0, 1, 1):
+                        continue
+                    for count_include_pad in [False, True]:
+                        # Skip "inclusive-counted blended or average pooling is not supported in combination with asymmetric padding"
+                        if count_include_pad and (padding == (0, 0, 1, 1) or strides == (2, 2)):
+                            continue
+                        run_and_verify(
+                            get_graph(
+                                relay.nn.avg_pool2d,
+                                pool_size=pool_size,
+                                strides=strides,
+                                padding=padding,
+                                ceil_mode=ceil_mode,
+                                count_include_pad=count_include_pad,
+                            )
+                        )
+                    run_and_verify(
+                        get_graph(
+                            relay.nn.max_pool2d,
+                            pool_size=pool_size,
+                            strides=strides,
+                            padding=padding,
+                            ceil_mode=ceil_mode,
+                        )
+                    )
+
+
+def test_global_pool2d():
+    def get_graph(op, x_shape=(1, 3, 32, 32)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.global_max_pool2d))
+    run_and_verify(get_graph(relay.nn.global_avg_pool2d))
+
+
+def test_batch_flatten():
+    def get_graph(x_shape=(1, 3, 4, 6)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.batch_flatten(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_expand_dims():
+    def get_graph(x_shape=(1, 3), axis=1, num_newaxis=1):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.expand_dims(x, axis, num_newaxis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_squeeze():
+    def get_graph(x_shape, axis):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.squeeze(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 5, 1, 1), (2, 3)))
+    run_and_verify(get_graph((1, 3, 1), (-1,)))
+
+
+def test_concatenate():
+    def get_graph(input_shapes, axis):
+        concat_inputs = []
+        shapes_dict = {}
+        for i in range(len(input_shapes)):
+            name = "input_{}".format(i)
+            concat_inputs.append(relay.var(name, shape=(input_shapes[i]), dtype="float32"))
+            shapes_dict[name] = input_shapes[i]
+        out = relay.concatenate(concat_inputs, axis)
+        f = relay.Function(concat_inputs, out)
+        return f, shapes_dict, []
+
+    run_and_verify(get_graph([(1, 2, 6, 6), (1, 3, 6, 6)], axis=1))
+
+
+def test_conv2d_transpose():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(32, 16, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d_transpose(
+            x,
+            kernel,
+            channels=k_shape[1],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    for padding in [(0, 0), (1, 1)]:
+        for strides in [(1, 1), (2, 2)]:
+            run_and_verify(get_graph(padding=padding, strides=strides))
+
+
+def test_reshape():
+    def get_graph(x_shape, new_shape):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.reshape(x, new_shape)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 1, 1, 10), (-1, 10)))
+    run_and_verify(get_graph((1, 10, 2, 3), (1, -1)))
+    run_and_verify(get_graph((1, 1, 2, 3), (1, 6)))
+
+
+def test_transpose():
+    def get_graph(x_shape, order):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.transpose(x, order)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 16, 7, 7), [0, 2, 3, 1]))
+    run_and_verify(get_graph((1, 7, 7, 16), [0, 3, 1, 2]))
+
+
+def test_float_const():
+    def get_graph(x_shape=(1, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        beta = relay.const(1, dtype="float32")
+        out = relay.multiply(x, beta)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_pad():
+    def get_graph(x_shape, pad_width):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.pad(x, pad_width=pad_width)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0]]))
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [1, 1], [1, 1]]))
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [0, 1], [2, 0]]))
+    run_and_verify(get_graph((1, 8, 3, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0], [0, 0]]))
+
+
+def test_softmax():
+    def get_graph(x_shape, axis):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.softmax(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 1000), axis=1))
+    run_and_verify(get_graph((1, 1000), axis=-1))
+    run_and_verify(get_graph((1, 3, 4), axis=-2))
+    run_and_verify(get_graph((1, 3, 4), axis=1))
+
+
+def test_batch_norm():
+    def get_graph(x_shape, param_shape, axis=1, epsilon=1e-5):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        beta = relay.var("beta", shape=(param_shape), dtype="float32")
+        gamma = relay.var("gamma", shape=(param_shape), dtype="float32")
+        moving_mean = relay.var("moving_mean", shape=(param_shape), dtype="float32")
+        moving_var = relay.var("moving_var", shape=(param_shape), dtype="float32")
+        out, _, _ = relay.nn.batch_norm(
+            x,
+            gamma=gamma,
+            beta=beta,
+            moving_mean=moving_mean,
+            moving_var=moving_var,
+            axis=axis,
+            center=True,
+            scale=True,
+            epsilon=epsilon,
+        )
+        f = relay.Function([x, gamma, beta, moving_mean, moving_var], out)
+        return (
+            f,
+            {
+                "x": x_shape,
+                "beta": param_shape,
+                "gamma": param_shape,
+                "moving_mean": param_shape,
+                "moving_var": param_shape,
+            },
+            ["beta", "gamma", "moving_mean", "moving_var"],
+        )
+
+    run_and_verify(get_graph((1, 64, 56, 56), (64,)))
+    run_and_verify(get_graph((1, 56, 56, 64), (64,), axis=3, epsilon=1.001e-05))
+
+
+def test_unary():
+    def get_graph(op, x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for op in [
+        relay.nn.relu,
+        relay.sigmoid,
+        relay.tanh,
+        relay.exp,
+        relay.log,
+        relay.sqrt,
+        relay.abs,
+        relay.negative,
+        relay.sin,
+        relay.cos,
+        relay.atan,
+        relay.ceil,
+        relay.floor,
+    ]:
+        run_and_verify(get_graph(op))
+
+
+def test_clip():
+    def get_graph(x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.clip(x, a_min=-0.2, a_max=0.4)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_leaky_relu():
+    def get_graph(x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.leaky_relu(x, alpha=0.1)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_binary():
+    def get_graph(op, x_shape, y_shape, y_is_const=False):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if y_is_const:
+            y = relay.const(np.ones(y_shape).astype("float32"))
+            out = op(x, y)
+            f = relay.Function([x], out)
+            return f, {"x": x_shape}, []
+        y = relay.var("y", shape=(y_shape), dtype="float32")
+        out = op(x, y)
+        f = relay.Function([x, y], out)
+        return f, {"x": x_shape, "y": y_shape}, []
+
+    for op in [relay.add, relay.subtract, relay.multiply, relay.divide, relay.power]:
+        for y_is_const in [True, False]:
+            run_and_verify(get_graph(op, (1, 8, 3, 3), (1, 8, 3, 3), y_is_const))
+            run_and_verify(get_graph(op, (1, 8, 1, 3), (1, 8, 3, 1), y_is_const))
+            run_and_verify(get_graph(op, (1, 10), (10,), y_is_const))
+            run_and_verify(get_graph(op, (1, 1, 1, 10), (10,), y_is_const))
+            run_and_verify(get_graph(op, (1, 1, 1), (3,), y_is_const))
+
+
+def test_reduce():
+    def get_graph(op, x_shape=(1, 2, 3, 4), axis=(2, 3), keepdims=False):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x, axis=axis, keepdims=keepdims)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for op in [relay.sum, relay.prod, relay.max, relay.min, relay.mean]:
+        for keepdims in [True, False]:
+            run_and_verify(get_graph(op, axis=(1), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(2, 3), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(1, 2), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(1, 2, 3), keepdims=keepdims))
+
+
+def test_strided_slice():
+    def get_graph(x_shape, begin, end, strides=None):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if strides:
+            out = relay.strided_slice(
+                x,
+                relay.expr.const(begin, dtype="int32"),
+                relay.expr.const(end, dtype="int32"),
+                relay.expr.const(strides, dtype="int32"),
+            )
+        else:
+            out = relay.strided_slice(
+                x,
+                relay.expr.const(begin, dtype="int32"),
+                relay.expr.const(end, dtype="int32"),
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 3, 6, 7), [0, 0, 0, 0], [1, 1, 6, 7]))
+    run_and_verify(get_graph((1, 3, 6, 7), [0, 1, 0, 0], [1, 2, 6, 6]))
+    run_and_verify(get_graph((1, 10), [0, 0], [1, 10], [1, 2]))
+
+
+def test_adaptive_pool2d():
+    def get_graph(op, x_shape=(1, 3, 32, 32), out_size=(1, 1)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x, out_size)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.adaptive_max_pool2d))
+    run_and_verify(get_graph(relay.nn.adaptive_avg_pool2d))
+
+
+def test_multiple_outputs():
+    def get_graph():
+        x = relay.var("x", shape=(1, 3), dtype="float32")
+        y = relay.var("y", shape=(1, 3), dtype="float32")
+        z = relay.add(x, y)
+        w = relay.add(z, y)
+        out = relay.Tuple((z, w))
+        f = relay.Function([x, y], out)
+        return f, {"x": (1, 3), "y": (1, 3)}, []
+
+    run_and_verify(get_graph())
+
+
+def test_conv3d():
+    def get_graph(
+        x_shape=(1, 32, 8, 8, 8),
+        k_shape=(16, 32, 3, 3, 3),
+        groups=1,
+        padding=(0, 0, 0),
+        strides=(1, 1, 1),
+        dilation=(1, 1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv3d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph(padding=(0, 0, 0, 1, 1, 1)))
+
+
+def test_pool3d():
+    def get_graph(
+        op,
+        x_shape=(1, 3, 8, 32, 32),
+        pool_size=(2, 2, 2),
+        strides=(2, 2, 2),
+        padding=(0, 0, 0),
+        ceil_mode=False,
+        count_include_pad=None,
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if count_include_pad is not None:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+                count_include_pad=count_include_pad,
+            )
+        else:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.avg_pool3d))
+    run_and_verify(get_graph(relay.nn.max_pool3d))
+    run_and_verify(get_graph(relay.nn.max_pool3d, padding=(0, 0, 0, 1, 1, 1)))
+    run_and_verify(get_graph(relay.nn.max_pool3d, strides=(1, 1, 1)))
+
+
+def test_conv3d_transpose():
+    def get_graph(
+        x_shape=(1, 32, 8, 8, 8),
+        k_shape=(32, 16, 3, 3, 3),
+        groups=1,
+        padding=(0, 0, 0),
+        strides=(1, 1, 1),
+        output_padding=(0, 0, 0),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv3d_transpose(
+            x,
+            kernel,
+            channels=k_shape[1],
+            kernel_size=k_shape[2:5],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            output_padding=output_padding,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph(strides=(2, 2, 2)))
+    run_and_verify(get_graph(strides=(2, 2, 2), output_padding=(1, 1, 1)))
+
+
+def test_alexnet():
+    run_and_verify_model("alexnet")
+
+
+def test_resnet18_v1():
+    run_and_verify_model("resnet18_v1")
+
+
+def test_resnet18_v2():
+    run_and_verify_model("resnet18_v2")
+
+
+def test_squeezenet():
+    run_and_verify_model("squeezenet1.0")
+
+
+def test_mobilenet():
+    run_and_verify_model("mobilenet0.25")
+
+
+def test_mobilenet_v2():
+    run_and_verify_model("mobilenetv2_0.25")
+
+
+def test_vgg11():
+    run_and_verify_model("vgg11")
+
+
+def test_densenet121():
+    run_and_verify_model("densenet121")

Review comment:
       I don't think we need to test that many models in the CI. Maybe one ResNet and one MobileNet would be sufficient.

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):

Review comment:
       It's unclear what this function does especially you have `run_and_verify_model`. Maybe `run_and_verify_func`?

##########
File path: src/relay/backend/contrib/tensorrt/codegen.cc
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/tensorrt/codegen.cc
+ * \brief Implementation of the TensorRT JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../utils.h"
+#include "../codegen_json/codegen_json.h"
+
+#if TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#endif
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*! \brief Attributes to store the compiler options for TensorRT. */
+struct TensorRTCompilerConfigNode : public tvm::AttrsNode<TensorRTCompilerConfigNode> {
+  Array<Integer> tensorrt_version;
+  bool use_implicit_batch;
+  size_t max_workspace_size;
+  bool remove_no_mac_subgraphs;
+
+  TVM_DECLARE_ATTRS(TensorRTCompilerConfigNode, "ext.attrs.TensorRTCompilerConfigNode") {
+    TVM_ATTR_FIELD(tensorrt_version)
+        .describe("TensorRT version as (major, minor, patch).")
+        .set_default(Array<Integer>({6, 0, 1}));
+    TVM_ATTR_FIELD(use_implicit_batch).set_default(true);
+    TVM_ATTR_FIELD(max_workspace_size).set_default(size_t(1) << 30);
+    TVM_ATTR_FIELD(remove_no_mac_subgraphs).set_default(false);
+  }
+};
+
+class TensorRTCompilerConfig : public Attrs {
+ public:
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TensorRTCompilerConfig, Attrs,
+                                            TensorRTCompilerConfigNode);
+};
+
+TVM_REGISTER_NODE_TYPE(TensorRTCompilerConfigNode);
+TVM_REGISTER_PASS_CONFIG_OPTION("relay.ext.tensorrt.options", TensorRTCompilerConfig);
+
+/*!
+ * \brief Generates an TensorRTModule from a relay expression by serializing the expression to a
+ * json representation. TensorRT is not required here because use of TensorRT APIs is deferred until
+ * runtime.
+ */
+class TensorRTJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  TensorRTJSONSerializer(const std::string& symbol, const Expr& expr)
+      : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+
+    std::vector<JSONGraphNodeEntry> inputs;
+    for (const auto& arg : cn->args) {
+      auto res = VisitExpr(arg);
+      inputs.insert(inputs.end(), res.begin(), res.end());
+    }
+    auto node = std::make_shared<JSONGraphNode>(name,     /* name_ */
+                                                "kernel", /* op_type_ */
+                                                inputs, 1 /* num_outputs_ */);
+    if (name == "nn.pad") {
+      SetPadNodeAttribute(node, cn);
+    } else if (name == "strided_slice") {
+      SetStridedSliceNodeAttribute(node, cn);
+    } else {
+      SetCallNodeAttribute(node, cn);
+    }
+    // These attributes are global to the whole module.
+    SaveGlobalAttributes(node);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  void SetPadNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* pad_attr = cn->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    const int dim_h = (p.size() == 5) ? 3 : 2;
+    const int dim_w = (p.size() == 5) ? 4 : 3;
+    std::vector<std::string> padding = {std::to_string(p[dim_h][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_h][1].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    node->SetAttr("padding", padding_attr);
+  }
+
+  void SetStridedSliceNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* attrs = cn->attrs.as<StridedSliceAttrs>();
+    CHECK(attrs);
+    CHECK(attrs->begin && attrs->end && attrs->strides);

Review comment:
       We can merge these two checks and provide a proper error message.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_utils.h
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/utils.h
+ * \brief Helper functions used by TensorRTBuilder or TrtOpConverters.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+
+#include <string>
+#include <vector>
+
+#include "NvInfer.h"
+
+// There is a conflict between cpplint and clang-format-10.
+// clang-format off
+#define TRT_VERSION_GE(major, minor, patch)                                                    \
+  ((NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
+  (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
+// clang-format on
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+/*!
+ * \brief Helper function to convert an vector to TRT Dims.
+ * \param vec Vector.
+ * \return TRT Dims.
+ */
+template <typename T>
+inline nvinfer1::Dims VectorToTrtDims(const std::vector<T>& vec) {
+  nvinfer1::Dims dims;
+  // Dims(nbDims=0, d[0]=1) is used to represent a scalar in TRT.
+  dims.d[0] = 1;
+  dims.nbDims = vec.size();
+  for (size_t i = 0; i < vec.size(); ++i) {
+    dims.d[i] = vec[i];
+  }
+  return dims;
+}
+
+/*!
+ * \brief Helper function to convert TRT Dims to vector.
+ * \param vec TRT Dims.
+ * \return Vector.
+ */
+inline std::vector<int> TrtDimsToVector(const nvinfer1::Dims& dims) {
+  return std::vector<int>(dims.d, dims.d + dims.nbDims);
+}
+
+/*!
+ * \brief Helper function to convert vector to string.
+ * \param vec Vector.
+ * \return Vector as a string.
+ */
+template <typename T>
+inline std::string DebugString(const std::vector<T>& vec) {

Review comment:
       @zhiics do we have existing helper functions to achieve the same goal?

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):
+    if skip_codegen_test():
+        return
+    f, input_shapes, is_param = config
+    params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+    input_dict = {
+        k: np.random.uniform(-1, 1, v).astype(np.float32)
+        for k, v in input_shapes.items()
+        if k not in is_param
+    }
+
+    # Run TRT
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    # Run reference
+    mod = tvm.IRModule()
+    mod["main"] = f
+    with tvm.transform.PassContext(opt_level=3):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    assert len(results) == len(ref_results)
+    for i in range(len(results)):
+        res = results[i].asnumpy()
+        ref_res = ref_results[i].asnumpy()
+        assert res.shape == ref_res.shape
+        tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+
+def run_and_verify_model(model):
+    if skip_codegen_test():
+        return
+
+    def compile_and_run(i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+
+        def check_trt_used(graph):
+            import json
+
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum(
+                [
+                    1
+                    for n in graph["nodes"]
+                    if n.get("attrs", {}).get("func_name", "").startswith("tensorrt_")
+                ]
+            )
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={"data": input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod, config = tensorrt.partition_for_tensorrt(mod, params)
+            with tvm.transform.PassContext(
+                opt_level=3, config={"relay.ext.tensorrt.options": config}
+            ):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+            check_trt_used(graph)
+        else:
+            with tvm.transform.PassContext(opt_level=3):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+
+        if skip_runtime_test():
+            return
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return res
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    res = compile_and_run(i_data, input_shape, dtype, use_trt=True)
+    ref_res = compile_and_run(i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+    tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)
+
+
+def test_tensorrt_simple():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3, 1, 1)
+    zshape = (1, 1, 1, 1)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.var("y", shape=(yshape), dtype=dtype)
+    z = relay.var("z", shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_not_compatible():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 32, 14, 14)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_serialize():
+    if skip_codegen_test():
+        return
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    block = get_model("resnet18_v1", pretrained=True)
+    mod, params = relay.frontend.from_mxnet(
+        block, shape={"data": (1, 3, 224, 224)}, dtype="float32"
+    )
+    # Compile
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        lib = relay.build(mod, "cuda", params=params)
+    # Serialize
+    lib.export_library("compiled.so")
+    # Deserialize
+    loaded_lib = tvm.runtime.load_module("compiled.so")
+    # Run
+    if skip_runtime_test():
+        return
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib["default"](tvm.gpu(0)))
+    i_data = np.random.uniform(0, 1, (1, 3, 224, 224)).astype("float32")
+    for i in range(10):
+        gen_module.run(data=i_data)

Review comment:
       Why you need to run 10 times?

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {

Review comment:
       Could you remind me what other possible types here?

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       This is an interesting discussion. I realized that this is more like a serialization for platform-dependent TensorRT engines. If it's not possible to build and serialize the engine during the compilation (or cross-compilation) even we have built the TVM with TensorRT runtime, then this is probably inevitable; otherwise we may build the engine and serialize the bit-stream along with other artifacts in `SaveToBinary`.
   
   If the serialization here is inevitable, which I believe in it because users may not have TensorRT during compilation, then the next question is whether we can update the ".so" file with the serialized engine here instead of creating a separate file. In other words, the .so file may or may not contain a serialized engine, but if it has, we don't need to build it again.
   
   
   
   




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

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



[GitHub] [incubator-tvm] areusch commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
areusch commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707311562


   @trevor-m I don't see anything weird with your build rules, but I wonder if changing the cmake config affected something. do you have >1 CI failure showing the segfault, or can you reproduce this locally?
   
   we have seen this before sporadically but don't know what causes it, and it's usually pretty hard to reproduce


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r488828728



##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,573 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+def should_skip():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("skip because cuda is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("skip because tensorrt runtime is not available")
+        return True
+    return False
+
+def test_tensorrt_simple():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3,  1,  1)
+    zshape = (1,  1,  1,  1)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.var('y', shape=(yshape), dtype=dtype)
+    z = relay.var('z', shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_not_compatible():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 32, 14, 14)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_ops():
+    if should_skip():
+        return
+    def run_and_verify(config):
+        f, input_shapes, is_param = config
+        params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+        input_dict = {k: np.random.uniform(-1, 1, v).astype(np.float32) for k, v in input_shapes.items() if k not in is_param}
+
+        # Run TRT 
+        mod = tvm.IRModule()
+        mod['main'] = f
+        mod = tensorrt.partition_for_tensorrt(mod, params)
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**graph_params)
+        mod.run(**input_dict)
+        results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+        # Run reference
+        mod = tvm.IRModule()
+        mod['main'] = f
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)

Review comment:
       I see, I can move all subtests from test_tensorrt_ops and test_tensorrt_integration into their own individual functions instead.




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   For the rest 2 points.
   
   2. Is that possible to move the pass before partitioning but after merge compiler region? After the merge compiler region pass you should get the Relay graph with almost the same semantic as partitioning. If you could have a pass checking each compiler region for your constraints, you can probably just remove the region you don't want, so that you should get only valid partitioned functions.
   
   3. Can the TensorRT version be obtained via an API call in C++? Something like `tensorrt::get_version()`? If so you can register a global symbol and pass the version to Python so that it can be used by the annotator.
   
   ```python
   def conv2d(...):
       if not tvm.get_global_func("relay.tensorrt.version", True):
           return False
       ver = tvm.get_global_func("relay.tensorrt.version")
       if ver == '1.0':
           return True
       return False
   ```
   
   If you need manually set up the TensorRT version, then it could be like this: Let user specify it in `config.cmake` and we pass the value to a macro in C++ so that you could simply return the value. The drawback of this solution is that it needs to rebuild TVM to annotate different TensorRT versions, and I'm not sure if that makes sense to 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.

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r500591165



##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,288 @@
+..  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.
+
+Relay TensorRT Integration
+==========================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+This guide will demonstrate how to install TensorRT and build TVM with TensorRT BYOC and runtime
+enabled. It will also provide example code to compile and run a ResNet-18 model using TensorRT and
+how to configure the compilation and runtime settings. Finally, we document the supported operators
+and how to extend the integration to support other operators.
+
+Installing TensorRT
+-------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.

Review comment:
       yeah, exactly




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------

Review comment:
       ```suggestion
   -------------------
   ```
   Note that misaligning will cause doc generation warnings and fail the CI.




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

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



[GitHub] [incubator-tvm] tqchen commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-708021897


   Notably we will need to wait until the docker image is updated instead just the PR merge. I believe @jroesch might be working on an image update that we can let him chime in once it lands. hopefully not blocking it for too long. We can also land and re-enable later


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r492407348



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       Thanks for the review Cody!
   
   You're right, the names aren't really that clear here. Originally, I had them as `USE_TENSORRT_CODEGEN` for codegen only and `USE_TENSORRT` for both codegen and runtime. I changed them to match the ACL definitions.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {

Review comment:
       There can also be weights when `GetOpType() == "const"`

##########
File path: src/runtime/contrib/tensorrt/tensorrt_ops.h
##########
@@ -0,0 +1,208 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_ops.h
+ * \brief Converters from Relay ops into TensorRT layers. Converters should
+ * inherit from TrtOpConverter and implement the Convert() method.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+
+#include <algorithm>
+#include <cmath>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_utils.h"
+
+#if TRT_VERSION_GE(6, 0, 1)
+#define TRT_HAS_IMPLICIT_BATCH(params) (params->network->hasImplicitBatchDimension())
+#else
+#define TRT_HAS_IMPLICIT_BATCH(params) (true)
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+
+/*!
+ * \brief An input to a op may be either kTensor in the case of nvinfer::ITensor*
+ * or kWeight for nvinfer1::Weights.
+ */
+enum TrtInputType {
+  kTensor,
+  kWeight,
+};
+
+/*!
+ * \brief An input to a TrtOpConverter. The type of the input is either kTensor
+ * or kWeight. For kTensor, "tensor" contains the input tensor. For kWeight,
+ * "weight" contains the input weight and "weight_shape" contains the shape.
+ */
+struct TrtOpInput {
+  /*! \brief If type is kTensor, will store input tensor. */
+  nvinfer1::ITensor* tensor;
+
+  /*! \brief If type is kWeight, will store input weight. */
+  nvinfer1::Weights weight;
+
+  /*! \brief Whether the input is in tensor or weight. */
+  TrtInputType type;
+
+  /*! \brief If type is kWeight, will store weight shape. */
+  std::vector<int> weight_shape;
+
+  explicit TrtOpInput(nvinfer1::ITensor* tensor)
+      : tensor(tensor), weight({nvinfer1::DataType::kFLOAT, nullptr, 0}), type(kTensor) {}
+  TrtOpInput(nvinfer1::Weights weight, const std::vector<int>& shape)
+      : tensor(nullptr), weight(weight), type(kWeight), weight_shape(shape) {}
+};
+
+/*! \brief Parameters to convert an Op from relay to TensorRT. */
+struct AddTrtLayerParams {

Review comment:
       Renamed to `TensorRTOpConverterParams`




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   > Thanks Cody, I measured the time on a AWS g4dn.4xlarge instance.
   > 
   > Full with runtime: 6m2.021s
   > Codegen only: 1m0.358s
   > Codegen only, removed all integration test models except resnet18_v1 and mobilenet_v2: 0m51.536s
   > 
   > Seems like the extra models in the integration tests do not consume much time.
   
   Thanks for the numbers. Ideally, we should include TensorRT in the CI, so 6 mins should be the number we are referring. @zhiics @tqchen please share your opinions about the testing time.
   


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       The message is a bit confusing. `USE_TENSORRT` means enabling the TensorRT codegen for graph partitininog. It doesn't require TensorRT to be available in the system environment. IIUC, maybe it's better to say "Build with TensorRT codegen", although I just found that "Build with Arm Compute Library" has the same issue.
   
   @lhutton1 could you also share your thoughts for this?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch

Review comment:
       Do we need a guard here as in L52?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):
+        """
+        Visit an expression while replacing vars according to var_map. Used by
+        SubgraphRemover/PruneSubgraphs to return a subgraph originally partitioned to TRT back to TVM.
+        """
+        def __init__(self, var_map):
+            ExprMutator.__init__(self)
+            self.var_map = var_map
+
+        def visit_var(self, var):
+            if var in self.var_map:
+                return self.var_map[var]
+            return super().visit_var(var)
+
+    class SubgraphRemover(ExprMutator):

Review comment:
       This is the issue we discussed in this PR about how to deal with post-partitioning judgements. We could later on figure out an approach to generalize this requirement.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]

Review comment:
       ```suggestion
       ret: Tuple[int, int, int]
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]

Review comment:
       ```suggestion
       version : Optional[Tuple[int, int, int]]
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.

Review comment:
       Shoud be fine in one line.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs

Review comment:
       Do we need a guard here as in L52?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper

Review comment:
       These two functions share the same logic so it would be better to merge them:
   
   ```python
   def _register_external_op_helper_with_checker(op_name, checker):
       # The body of _register_external_op_helper_func
   
   def _register_external_op_helper(op_name, supported=True):
       return _register_external_op_helper_with_checker(op_name, lambda attrs, args, op_name: supported)
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.

Review comment:
       ```suggestion
       mod_n_config: Tuple[Module, Dict[str, Any]]
           A tuple of 1) annotated and partitioned module and 2) "relay.ext.tensorrt.options" configuration
           which should be given to PassContext when building.
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")

Review comment:
       - Use the standard logger: `logger = logging.getLogger("TensorRT")`.
   - "info" seems not a right log level for messages like this. Should be "warn" or "debug".
   - Ditto to other logs.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.

Review comment:
       s/relay/Relay/g

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================

Review comment:
       ```suggestion
   ==========================
   ```

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.

Review comment:
       Better to provide an overview of this tutorial, including what information/takeaway readers could expect after reading this article.
   
   After reading through, I would suggest mentioning the following points:
   1. (optional) Some model performance numbers to impressive readers.
   2. Install TensorRT and build TVM with it.
   3. Build a ResNet-18 with TensorRT.
   4. Operator support (I also suggest moving this section to the end along with the last section about adding new operators.)
   5. Deploy the built model with TensorRT runtime.
   6. How to support other operators.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------

Review comment:
       ```suggestion
   -------------------
   ```
   Note that misaligning will cause doc generation warnings and failed the CI.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.

Review comment:
       Better to say this section is tested with TensorRT version XXX and may not stay up-to-date with newer versions that changes the APIs.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (
+        not get_tensorrt_use_implicit_batch_mode()
+        and (isinstance(args[0], Constant) or isinstance(args[1], Constant))
+        and args[0].checked_type.shape[0] == args[1].checked_type.shape[0]
+        and args[0].checked_type.shape[0] != 1
+        and (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3)
+    ):
+        logging.info("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.batch_norm is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        logging.info("nn.batch_norm: axis is %d but must be 1 or 3.", int(attrs.axis))
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.softmax is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info("nn.conv2d: kernel_layout is %s but must be OIHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if dense is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.dense: input has rank %d but must be 2, 3 or 4.", input_rank)
+        return False
+    if weight_rank != 2:
+        logging.info("nn.dense: weight has rank %d but must be 2.", weight_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.bias_add is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.bias_add: input rank is %d but must be 2, 3 or 4.", input_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.avg_pool2d: layout is %d but must be NCHW.", attrs.layout)
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        logging.info(
+            "nn.avg_pool2d: inclusive-counted blended or average "
+            "pooling is not supported in combination with asymmetric padding"
+        )
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_avg_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if expand_dims is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if squeeze is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        logging.info("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        logging.info("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if concatenate is supported by TensorRT."""
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        logging.info("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                logging.info("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info(
+            "nn.conv2d_transpose: kernel_layout is %s but must be OIHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        logging.info("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def layout_transform_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if layout_transform is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [
+        ("NCHW", "NHWC"),
+        ("NHWC", "NCHW"),
+        ("NDHWC", "NCDHW"),
+        ("NCDHW", "NDHWC"),
+    ]:
+        logging.info(
+            "layout_transform: %s to %s is not supported.", attrs.src_layout, attrs.dst_layout
+        )
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if reshape is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        logging.info("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            logging.info("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            logging.info("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.pad is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        logging.info("nn.pad: pad mode is %s but must be constant.", attrs.pad_mode)
+        return False
+    if float(attrs.pad_value) != 0.0:
+        logging.info("nn.pad: pad value is %f but must be 0.0.", float(attrs.pad_value))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        logging.info("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        logging.info("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+
+def reduce_annotate_fn(attrs, args, op_name):
+    """Helper for reduce operations."""
+    if not attrs.axis or len(attrs.axis) == 0:
+        logging.info("%s: cannot reduce to scalar.", op_name)
+        return False
+    if attrs.exclude:
+        logging.info("%s: exclude not supported.", op_name)
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        logging.info("%s: can't modify batch dimension.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    """Helper for ops which require TRT 5.1.5 or greater."""
+    if get_tensorrt_version() < (5, 1, 5):
+        logging.info("%s: requires TensorRT version 5.1.5 or higher.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if strided_slice is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = (
+            attrs.end[0] is not None
+            and int(attrs.end[0]) != -1
+            and int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        )
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            logging.info("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        logging.info("strided_slice: stride must be positive")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):

Review comment:
       It might be better to generalize `trt_5_1_5_annotate_fn` to `trt_version_annotate_fn(version, attrs, args, op_name)`.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be "cuda".

Review comment:
       This statement is not 100% correct. It can also be "cuda -model=v100" for example.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage

Review comment:
       Explicitly say what you are doing in this section, such as "Build and Deploy ResNet-18 with TensorRT"

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (
+        not get_tensorrt_use_implicit_batch_mode()
+        and (isinstance(args[0], Constant) or isinstance(args[1], Constant))
+        and args[0].checked_type.shape[0] == args[1].checked_type.shape[0]
+        and args[0].checked_type.shape[0] != 1
+        and (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3)
+    ):
+        logging.info("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.batch_norm is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        logging.info("nn.batch_norm: axis is %d but must be 1 or 3.", int(attrs.axis))
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.softmax is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info("nn.conv2d: kernel_layout is %s but must be OIHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if dense is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.dense: input has rank %d but must be 2, 3 or 4.", input_rank)
+        return False
+    if weight_rank != 2:
+        logging.info("nn.dense: weight has rank %d but must be 2.", weight_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.bias_add is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.bias_add: input rank is %d but must be 2, 3 or 4.", input_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.avg_pool2d: layout is %d but must be NCHW.", attrs.layout)
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        logging.info(
+            "nn.avg_pool2d: inclusive-counted blended or average "
+            "pooling is not supported in combination with asymmetric padding"
+        )
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_avg_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if expand_dims is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if squeeze is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        logging.info("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        logging.info("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if concatenate is supported by TensorRT."""
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        logging.info("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                logging.info("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info(
+            "nn.conv2d_transpose: kernel_layout is %s but must be OIHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        logging.info("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def layout_transform_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if layout_transform is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [
+        ("NCHW", "NHWC"),
+        ("NHWC", "NCHW"),
+        ("NDHWC", "NCDHW"),
+        ("NCDHW", "NDHWC"),
+    ]:
+        logging.info(
+            "layout_transform: %s to %s is not supported.", attrs.src_layout, attrs.dst_layout
+        )
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if reshape is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        logging.info("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            logging.info("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            logging.info("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.pad is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        logging.info("nn.pad: pad mode is %s but must be constant.", attrs.pad_mode)
+        return False
+    if float(attrs.pad_value) != 0.0:
+        logging.info("nn.pad: pad value is %f but must be 0.0.", float(attrs.pad_value))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        logging.info("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        logging.info("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+
+def reduce_annotate_fn(attrs, args, op_name):
+    """Helper for reduce operations."""
+    if not attrs.axis or len(attrs.axis) == 0:
+        logging.info("%s: cannot reduce to scalar.", op_name)
+        return False
+    if attrs.exclude:
+        logging.info("%s: exclude not supported.", op_name)
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        logging.info("%s: can't modify batch dimension.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    """Helper for ops which require TRT 5.1.5 or greater."""
+    if get_tensorrt_version() < (5, 1, 5):
+        logging.info("%s: requires TensorRT version 5.1.5 or higher.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if strided_slice is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = (
+            attrs.end[0] is not None
+            and int(attrs.end[0]) != -1
+            and int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        )
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            logging.info("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        logging.info("strided_slice: stride must be positive")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        logging.info("nn.conv3d: data_layout is %s but must be NCDHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        logging.info("nn.conv3d: kernel_layout is %s but must be OIDHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        logging.info("nn.conv3d: out_layout is %s but must be NCDHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        logging.info("nn.max_pool3d: layout is %s but must be NCDHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        logging.info("nn.avg_pool3d: layout is %s but must be NCDHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        logging.info("nn.conv3d_transpose: data_layout is %s but must be NCDHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        logging.info(
+            "nn.conv3d_transpose: kernel_layout is %s but must be OIDHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        logging.info("nn.conv3d_transpose: out_layout is %s but must be NCDHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        logging.info("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+
+def is_valid_subgraph(params, body):
+    """Final check on whether the subgraph is valid and should be offloaded to TensorRT."""
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        logging.info("tensorrt: scalar inputs not supported")
+                        return False
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    logging.info("tensorrt: scalar inputs not supported")
+                    return False
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and len(set(input_batch_sizes)) != 1:
+            logging.info("tensorrt: inputs have different batch sizes")
+            return False
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return False
+    return True
+
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):

Review comment:
       As it is already prune_"tensorrt"_subgraphs, it looks to me that you don't need to specify `target` in the arguments?

##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       The message is a bit confusing. `USE_TENSORRT` means enabling the TensorRT codegen for graph partitininog. It doesn't require TensorRT to be available in the system environment. IIUC, maybe it's better to say "Build with TensorRT codegen", although I just found that "Build with Arm Compute Library" has the same issue.
   
   @lhutton1 could you also share your thoughts about this?

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------

Review comment:
       ```suggestion
   -------------------
   ```
   Note that misaligning will cause doc generation warnings and fail the CI.

##########
File path: src/relay/backend/contrib/tensorrt/codegen.cc
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/tensorrt/codegen.cc
+ * \brief Implementation of the TensorRT JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../utils.h"
+#include "../codegen_json/codegen_json.h"
+
+#if TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#endif
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*! \brief Attributes to store the compiler options for TensorRT. */
+struct TensorRTCompilerConfigNode : public tvm::AttrsNode<TensorRTCompilerConfigNode> {
+  Array<Integer> tensorrt_version;
+  bool use_implicit_batch;
+  size_t max_workspace_size;
+  bool remove_no_mac_subgraphs;
+
+  TVM_DECLARE_ATTRS(TensorRTCompilerConfigNode, "ext.attrs.TensorRTCompilerConfigNode") {
+    TVM_ATTR_FIELD(tensorrt_version)
+        .describe("TensorRT version as (major, minor, patch).")
+        .set_default(Array<Integer>({6, 0, 1}));
+    TVM_ATTR_FIELD(use_implicit_batch).set_default(true);
+    TVM_ATTR_FIELD(max_workspace_size).set_default(size_t(1) << 30);
+    TVM_ATTR_FIELD(remove_no_mac_subgraphs).set_default(false);
+  }
+};
+
+class TensorRTCompilerConfig : public Attrs {
+ public:
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TensorRTCompilerConfig, Attrs,
+                                            TensorRTCompilerConfigNode);
+};
+
+TVM_REGISTER_NODE_TYPE(TensorRTCompilerConfigNode);
+TVM_REGISTER_PASS_CONFIG_OPTION("relay.ext.tensorrt.options", TensorRTCompilerConfig);
+
+/*!
+ * \brief Generates an TensorRTModule from a relay expression by serializing the expression to a
+ * json representation. TensorRT is not required here because use of TensorRT APIs is deferred until
+ * runtime.
+ */
+class TensorRTJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  TensorRTJSONSerializer(const std::string& symbol, const Expr& expr)
+      : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+
+    std::vector<JSONGraphNodeEntry> inputs;
+    for (const auto& arg : cn->args) {
+      auto res = VisitExpr(arg);
+      inputs.insert(inputs.end(), res.begin(), res.end());
+    }
+    auto node = std::make_shared<JSONGraphNode>(name,     /* name_ */
+                                                "kernel", /* op_type_ */
+                                                inputs, 1 /* num_outputs_ */);
+    if (name == "nn.pad") {
+      SetPadNodeAttribute(node, cn);
+    } else if (name == "strided_slice") {
+      SetStridedSliceNodeAttribute(node, cn);
+    } else {
+      SetCallNodeAttribute(node, cn);
+    }
+    // These attributes are global to the whole module.
+    SaveGlobalAttributes(node);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  void SetPadNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* pad_attr = cn->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    const int dim_h = (p.size() == 5) ? 3 : 2;
+    const int dim_w = (p.size() == 5) ? 4 : 3;
+    std::vector<std::string> padding = {std::to_string(p[dim_h][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_h][1].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    node->SetAttr("padding", padding_attr);
+  }
+
+  void SetStridedSliceNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* attrs = cn->attrs.as<StridedSliceAttrs>();
+    CHECK(attrs);
+    CHECK(attrs->begin && attrs->end && attrs->strides);
+    const bool default_strides =
+        !attrs->strides.value().defined() || attrs->strides.value().size() == 0;
+    auto ishape = backend::GetShape(cn->args[0]->checked_type());
+
+    auto process_slice_index = [](Integer x, int default_value, int dim_value) {
+      if (!x.defined()) return default_value;
+      int value = x.as<IntImmNode>()->value;
+      if (value < 0) value += dim_value;
+      return value;
+    };
+
+    std::vector<std::string> start, size, strides;
+    for (size_t i = 0; i < attrs->begin.value().size(); ++i) {
+      const int begin_value = process_slice_index(attrs->begin.value()[i], 0, ishape[i]);
+      const int end_value = process_slice_index(attrs->end.value()[i], ishape[i], ishape[i]);
+      const int stride_value = (default_strides || i >= attrs->strides.value().size() ||
+                                !attrs->strides.value()[i].defined())
+                                   ? 1
+                                   : attrs->strides.value()[i].as<IntImmNode>()->value;
+      CHECK_GT(stride_value, 0);
+      const int size_value = (end_value - begin_value + stride_value - 1) / stride_value;
+      CHECK_GE(begin_value, 0);
+      CHECK_GT(size_value, 0);
+      start.push_back(std::to_string(begin_value));
+      size.push_back(std::to_string(size_value));
+      strides.push_back(std::to_string(stride_value));
+    }
+    std::vector<dmlc::any> start_attr, size_attr, strides_attr;
+    start_attr.emplace_back(start);
+    size_attr.emplace_back(size);
+    strides_attr.emplace_back(strides);
+    node->SetAttr("start", start_attr);
+    node->SetAttr("size", size_attr);
+    node->SetAttr("strides", strides_attr);
+  }
+
+  void SaveGlobalAttributes(std::shared_ptr<JSONGraphNode> node) {
+    auto ctx = transform::PassContext::Current();
+    auto cfg = ctx->GetConfig<TensorRTCompilerConfig>("relay.ext.tensorrt.options");
+    if (!cfg.defined()) {
+      cfg = AttrsWithDefaultValues<TensorRTCompilerConfig>();
+    }
+    CHECK_EQ(cfg.value()->tensorrt_version.size(), 3);
+    std::vector<std::string> tensorrt_version = {std::to_string(cfg.value()->tensorrt_version[0]),
+                                                 std::to_string(cfg.value()->tensorrt_version[1]),
+                                                 std::to_string(cfg.value()->tensorrt_version[2])};
+    std::vector<std::string> use_implicit_batch = {std::to_string(cfg.value()->use_implicit_batch)};
+    std::vector<std::string> max_workspace_size = {std::to_string(cfg.value()->max_workspace_size)};
+    std::vector<dmlc::any> tensorrt_version_attr, use_implicit_batch_attr, max_workspace_size_attr;
+    tensorrt_version_attr.emplace_back(tensorrt_version);
+    use_implicit_batch_attr.emplace_back(use_implicit_batch);
+    max_workspace_size_attr.emplace_back(max_workspace_size);
+    node->SetAttr("tensorrt_version", tensorrt_version_attr);
+    node->SetAttr("use_implicit_batch", use_implicit_batch_attr);
+    node->SetAttr("max_workspace_size", max_workspace_size_attr);
+  }
+};
+
+/*!
+ * \brief Create a runtime module for TensorRT.
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module TensorRTCompiler(const ObjectRef& ref) {
+  CHECK(ref->IsInstance<FunctionNode>()) << "The input ref is expected to be a Relay function.";
+  Function func = Downcast<Function>(ref);
+  std::string func_name = backend::GetExtSymbol(func);
+
+  TensorRTJSONSerializer serializer(func_name, func);
+  serializer.serialize();
+  std::string graph_json = serializer.GetJSON();
+  auto param_names = serializer.GetParams();
+  const auto* pf = runtime::Registry::Get("runtime.tensorrt_runtime_create");
+  CHECK(pf != nullptr) << "Cannot find JSON runtime module to create";

Review comment:
       Improve the error message to be more TensorRT specific.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_builder.h
##########
@@ -0,0 +1,159 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_builder.h
+ * \brief The TensorRTBuilder class can be used to convert a JSONRuntime graph into a TRT engine
+ * which can be used for inference.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_logger.h"
+#include "tensorrt_ops.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+/*!
+ * \brief The product of TensorRTBuilder which provides everything needed to
+ * perform inference.
+ */
+struct TrtEngineAndContext {

Review comment:
       Better to be consistent with `TensorRTBuilder`: `TensorRTEngineAndContext`.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_ops.h
##########
@@ -0,0 +1,208 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_ops.h
+ * \brief Converters from Relay ops into TensorRT layers. Converters should
+ * inherit from TrtOpConverter and implement the Convert() method.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+
+#include <algorithm>
+#include <cmath>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_utils.h"
+
+#if TRT_VERSION_GE(6, 0, 1)
+#define TRT_HAS_IMPLICIT_BATCH(params) (params->network->hasImplicitBatchDimension())
+#else
+#define TRT_HAS_IMPLICIT_BATCH(params) (true)
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+
+/*!
+ * \brief An input to a op may be either kTensor in the case of nvinfer::ITensor*
+ * or kWeight for nvinfer1::Weights.
+ */
+enum TrtInputType {
+  kTensor,
+  kWeight,
+};
+
+/*!
+ * \brief An input to a TrtOpConverter. The type of the input is either kTensor
+ * or kWeight. For kTensor, "tensor" contains the input tensor. For kWeight,
+ * "weight" contains the input weight and "weight_shape" contains the shape.
+ */
+struct TrtOpInput {
+  /*! \brief If type is kTensor, will store input tensor. */
+  nvinfer1::ITensor* tensor;
+
+  /*! \brief If type is kWeight, will store input weight. */
+  nvinfer1::Weights weight;
+
+  /*! \brief Whether the input is in tensor or weight. */
+  TrtInputType type;
+
+  /*! \brief If type is kWeight, will store weight shape. */
+  std::vector<int> weight_shape;
+
+  explicit TrtOpInput(nvinfer1::ITensor* tensor)
+      : tensor(tensor), weight({nvinfer1::DataType::kFLOAT, nullptr, 0}), type(kTensor) {}
+  TrtOpInput(nvinfer1::Weights weight, const std::vector<int>& shape)
+      : tensor(nullptr), weight(weight), type(kWeight), weight_shape(shape) {}
+};
+
+/*! \brief Parameters to convert an Op from relay to TensorRT. */
+struct AddTrtLayerParams {

Review comment:
       What does "Add" means here?

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):
+    if skip_codegen_test():
+        return
+    f, input_shapes, is_param = config
+    params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+    input_dict = {
+        k: np.random.uniform(-1, 1, v).astype(np.float32)
+        for k, v in input_shapes.items()
+        if k not in is_param
+    }
+
+    # Run TRT
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    # Run reference
+    mod = tvm.IRModule()
+    mod["main"] = f
+    with tvm.transform.PassContext(opt_level=3):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    assert len(results) == len(ref_results)
+    for i in range(len(results)):
+        res = results[i].asnumpy()
+        ref_res = ref_results[i].asnumpy()
+        assert res.shape == ref_res.shape
+        tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+
+def run_and_verify_model(model):
+    if skip_codegen_test():
+        return
+
+    def compile_and_run(i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+
+        def check_trt_used(graph):
+            import json
+
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum(
+                [
+                    1
+                    for n in graph["nodes"]
+                    if n.get("attrs", {}).get("func_name", "").startswith("tensorrt_")
+                ]
+            )
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={"data": input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod, config = tensorrt.partition_for_tensorrt(mod, params)
+            with tvm.transform.PassContext(
+                opt_level=3, config={"relay.ext.tensorrt.options": config}
+            ):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+            check_trt_used(graph)
+        else:
+            with tvm.transform.PassContext(opt_level=3):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+
+        if skip_runtime_test():
+            return
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return res
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    res = compile_and_run(i_data, input_shape, dtype, use_trt=True)
+    ref_res = compile_and_run(i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+    tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)
+
+
+def test_tensorrt_simple():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3, 1, 1)
+    zshape = (1, 1, 1, 1)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.var("y", shape=(yshape), dtype=dtype)
+    z = relay.var("z", shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_not_compatible():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 32, 14, 14)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_serialize():
+    if skip_codegen_test():
+        return
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    block = get_model("resnet18_v1", pretrained=True)
+    mod, params = relay.frontend.from_mxnet(
+        block, shape={"data": (1, 3, 224, 224)}, dtype="float32"
+    )
+    # Compile
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        lib = relay.build(mod, "cuda", params=params)
+    # Serialize
+    lib.export_library("compiled.so")
+    # Deserialize
+    loaded_lib = tvm.runtime.load_module("compiled.so")
+    # Run
+    if skip_runtime_test():
+        return
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib["default"](tvm.gpu(0)))
+    i_data = np.random.uniform(0, 1, (1, 3, 224, 224)).astype("float32")
+    for i in range(10):
+        gen_module.run(data=i_data)
+
+
+def test_conv2d():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(16, 32, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+        dilation=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    for k_shape, groups in [((16, 32, 3, 3), 1), ((32, 1, 3, 3), 32)]:
+        for padding in [(0, 0), (1, 1)]:
+            for strides in [(1, 1), (2, 2)]:
+                for dilation in [(1, 1), (2, 2)]:
+                    run_and_verify(
+                        get_graph(
+                            k_shape=k_shape,
+                            groups=groups,
+                            padding=padding,
+                            strides=strides,
+                            dilation=dilation,
+                        )
+                    )
+
+
+def test_conv2d_nhwc():
+    def get_graph(x_shape=(1, 8, 8, 32), k_shape=(3, 3, 32, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=16,
+            kernel_size=(3, 3),
+            data_layout="NHWC",
+            kernel_layout="HWIO",
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_conv2d_weights_const():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(16, 32, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+        dilation=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.const(np.ones(k_shape).astype("float32"))
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_conv2d_weights_transposed():
+    def get_graph(x_shape=(1, 32, 9, 9), k_shape=(3, 3, 32, 16), order=(3, 2, 0, 1)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        kernel_t = relay.transpose(kernel, order)
+        # Conv2d requires constant weights in TensorRT, so the weights should be transposed by
+        # FoldConstant.
+        out = relay.nn.conv2d(x, kernel_t, channels=k_shape[order[0]], kernel_size=(3, 3))
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_dense():
+    def get_graph(x_shape=(1, 16), k_shape=(32, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        # Dense requires constant weights in TensorRT, so the weights are transposed by us.
+        out = relay.nn.dense(x, kernel, units=k_shape[0])
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_bias_add():
+    def get_graph(x_shape=(1, 16), channels=16):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        bias = relay.var("bias", shape=(channels,), dtype="float32")
+        out = relay.nn.bias_add(x, bias)
+        f = relay.Function([x, bias], out)
+        return f, {"x": x_shape, "bias": (channels,)}, ["bias"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph((1, 6, 3, 4), 6))
+
+
+def test_pool2d():
+    def get_graph(
+        op,
+        x_shape=(1, 3, 32, 32),
+        pool_size=(2, 2),
+        strides=(2, 2),
+        padding=(0, 0),
+        ceil_mode=False,
+        count_include_pad=None,
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if count_include_pad is not None:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+                count_include_pad=count_include_pad,
+            )
+        else:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for pool_size in [(2, 2), (3, 3)]:
+        for strides in [(1, 1), (2, 2)]:
+            for padding in [(0, 0), (1, 1), (0, 0, 1, 1)]:
+                for ceil_mode in [False, True]:
+                    # Skip "the padding size is larger than or equal to the filter size for exclusive-counting pooling"
+                    if pool_size == (2, 2) and padding == (0, 0, 1, 1):
+                        continue
+                    for count_include_pad in [False, True]:
+                        # Skip "inclusive-counted blended or average pooling is not supported in combination with asymmetric padding"
+                        if count_include_pad and (padding == (0, 0, 1, 1) or strides == (2, 2)):
+                            continue
+                        run_and_verify(
+                            get_graph(
+                                relay.nn.avg_pool2d,
+                                pool_size=pool_size,
+                                strides=strides,
+                                padding=padding,
+                                ceil_mode=ceil_mode,
+                                count_include_pad=count_include_pad,
+                            )
+                        )
+                    run_and_verify(
+                        get_graph(
+                            relay.nn.max_pool2d,
+                            pool_size=pool_size,
+                            strides=strides,
+                            padding=padding,
+                            ceil_mode=ceil_mode,
+                        )
+                    )
+
+
+def test_global_pool2d():
+    def get_graph(op, x_shape=(1, 3, 32, 32)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.global_max_pool2d))
+    run_and_verify(get_graph(relay.nn.global_avg_pool2d))
+
+
+def test_batch_flatten():
+    def get_graph(x_shape=(1, 3, 4, 6)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.batch_flatten(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_expand_dims():
+    def get_graph(x_shape=(1, 3), axis=1, num_newaxis=1):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.expand_dims(x, axis, num_newaxis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_squeeze():
+    def get_graph(x_shape, axis):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.squeeze(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 5, 1, 1), (2, 3)))
+    run_and_verify(get_graph((1, 3, 1), (-1,)))
+
+
+def test_concatenate():
+    def get_graph(input_shapes, axis):
+        concat_inputs = []
+        shapes_dict = {}
+        for i in range(len(input_shapes)):
+            name = "input_{}".format(i)
+            concat_inputs.append(relay.var(name, shape=(input_shapes[i]), dtype="float32"))
+            shapes_dict[name] = input_shapes[i]
+        out = relay.concatenate(concat_inputs, axis)
+        f = relay.Function(concat_inputs, out)
+        return f, shapes_dict, []
+
+    run_and_verify(get_graph([(1, 2, 6, 6), (1, 3, 6, 6)], axis=1))
+
+
+def test_conv2d_transpose():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(32, 16, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d_transpose(
+            x,
+            kernel,
+            channels=k_shape[1],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    for padding in [(0, 0), (1, 1)]:
+        for strides in [(1, 1), (2, 2)]:
+            run_and_verify(get_graph(padding=padding, strides=strides))
+
+
+def test_reshape():
+    def get_graph(x_shape, new_shape):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.reshape(x, new_shape)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 1, 1, 10), (-1, 10)))
+    run_and_verify(get_graph((1, 10, 2, 3), (1, -1)))
+    run_and_verify(get_graph((1, 1, 2, 3), (1, 6)))
+
+
+def test_transpose():
+    def get_graph(x_shape, order):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.transpose(x, order)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 16, 7, 7), [0, 2, 3, 1]))
+    run_and_verify(get_graph((1, 7, 7, 16), [0, 3, 1, 2]))
+
+
+def test_float_const():
+    def get_graph(x_shape=(1, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        beta = relay.const(1, dtype="float32")
+        out = relay.multiply(x, beta)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_pad():
+    def get_graph(x_shape, pad_width):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.pad(x, pad_width=pad_width)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0]]))
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [1, 1], [1, 1]]))
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [0, 1], [2, 0]]))
+    run_and_verify(get_graph((1, 8, 3, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0], [0, 0]]))
+
+
+def test_softmax():
+    def get_graph(x_shape, axis):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.softmax(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 1000), axis=1))
+    run_and_verify(get_graph((1, 1000), axis=-1))
+    run_and_verify(get_graph((1, 3, 4), axis=-2))
+    run_and_verify(get_graph((1, 3, 4), axis=1))
+
+
+def test_batch_norm():
+    def get_graph(x_shape, param_shape, axis=1, epsilon=1e-5):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        beta = relay.var("beta", shape=(param_shape), dtype="float32")
+        gamma = relay.var("gamma", shape=(param_shape), dtype="float32")
+        moving_mean = relay.var("moving_mean", shape=(param_shape), dtype="float32")
+        moving_var = relay.var("moving_var", shape=(param_shape), dtype="float32")
+        out, _, _ = relay.nn.batch_norm(
+            x,
+            gamma=gamma,
+            beta=beta,
+            moving_mean=moving_mean,
+            moving_var=moving_var,
+            axis=axis,
+            center=True,
+            scale=True,
+            epsilon=epsilon,
+        )
+        f = relay.Function([x, gamma, beta, moving_mean, moving_var], out)
+        return (
+            f,
+            {
+                "x": x_shape,
+                "beta": param_shape,
+                "gamma": param_shape,
+                "moving_mean": param_shape,
+                "moving_var": param_shape,
+            },
+            ["beta", "gamma", "moving_mean", "moving_var"],
+        )
+
+    run_and_verify(get_graph((1, 64, 56, 56), (64,)))
+    run_and_verify(get_graph((1, 56, 56, 64), (64,), axis=3, epsilon=1.001e-05))
+
+
+def test_unary():
+    def get_graph(op, x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for op in [
+        relay.nn.relu,
+        relay.sigmoid,
+        relay.tanh,
+        relay.exp,
+        relay.log,
+        relay.sqrt,
+        relay.abs,
+        relay.negative,
+        relay.sin,
+        relay.cos,
+        relay.atan,
+        relay.ceil,
+        relay.floor,
+    ]:
+        run_and_verify(get_graph(op))
+
+
+def test_clip():
+    def get_graph(x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.clip(x, a_min=-0.2, a_max=0.4)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_leaky_relu():
+    def get_graph(x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.leaky_relu(x, alpha=0.1)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_binary():
+    def get_graph(op, x_shape, y_shape, y_is_const=False):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if y_is_const:
+            y = relay.const(np.ones(y_shape).astype("float32"))
+            out = op(x, y)
+            f = relay.Function([x], out)
+            return f, {"x": x_shape}, []
+        y = relay.var("y", shape=(y_shape), dtype="float32")
+        out = op(x, y)
+        f = relay.Function([x, y], out)
+        return f, {"x": x_shape, "y": y_shape}, []
+
+    for op in [relay.add, relay.subtract, relay.multiply, relay.divide, relay.power]:
+        for y_is_const in [True, False]:
+            run_and_verify(get_graph(op, (1, 8, 3, 3), (1, 8, 3, 3), y_is_const))
+            run_and_verify(get_graph(op, (1, 8, 1, 3), (1, 8, 3, 1), y_is_const))
+            run_and_verify(get_graph(op, (1, 10), (10,), y_is_const))
+            run_and_verify(get_graph(op, (1, 1, 1, 10), (10,), y_is_const))
+            run_and_verify(get_graph(op, (1, 1, 1), (3,), y_is_const))
+
+
+def test_reduce():
+    def get_graph(op, x_shape=(1, 2, 3, 4), axis=(2, 3), keepdims=False):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x, axis=axis, keepdims=keepdims)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for op in [relay.sum, relay.prod, relay.max, relay.min, relay.mean]:
+        for keepdims in [True, False]:
+            run_and_verify(get_graph(op, axis=(1), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(2, 3), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(1, 2), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(1, 2, 3), keepdims=keepdims))
+
+
+def test_strided_slice():
+    def get_graph(x_shape, begin, end, strides=None):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if strides:
+            out = relay.strided_slice(
+                x,
+                relay.expr.const(begin, dtype="int32"),
+                relay.expr.const(end, dtype="int32"),
+                relay.expr.const(strides, dtype="int32"),
+            )
+        else:
+            out = relay.strided_slice(
+                x,
+                relay.expr.const(begin, dtype="int32"),
+                relay.expr.const(end, dtype="int32"),
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 3, 6, 7), [0, 0, 0, 0], [1, 1, 6, 7]))
+    run_and_verify(get_graph((1, 3, 6, 7), [0, 1, 0, 0], [1, 2, 6, 6]))
+    run_and_verify(get_graph((1, 10), [0, 0], [1, 10], [1, 2]))
+
+
+def test_adaptive_pool2d():
+    def get_graph(op, x_shape=(1, 3, 32, 32), out_size=(1, 1)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x, out_size)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.adaptive_max_pool2d))
+    run_and_verify(get_graph(relay.nn.adaptive_avg_pool2d))
+
+
+def test_multiple_outputs():
+    def get_graph():
+        x = relay.var("x", shape=(1, 3), dtype="float32")
+        y = relay.var("y", shape=(1, 3), dtype="float32")
+        z = relay.add(x, y)
+        w = relay.add(z, y)
+        out = relay.Tuple((z, w))
+        f = relay.Function([x, y], out)
+        return f, {"x": (1, 3), "y": (1, 3)}, []
+
+    run_and_verify(get_graph())
+
+
+def test_conv3d():
+    def get_graph(
+        x_shape=(1, 32, 8, 8, 8),
+        k_shape=(16, 32, 3, 3, 3),
+        groups=1,
+        padding=(0, 0, 0),
+        strides=(1, 1, 1),
+        dilation=(1, 1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv3d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph(padding=(0, 0, 0, 1, 1, 1)))
+
+
+def test_pool3d():
+    def get_graph(
+        op,
+        x_shape=(1, 3, 8, 32, 32),
+        pool_size=(2, 2, 2),
+        strides=(2, 2, 2),
+        padding=(0, 0, 0),
+        ceil_mode=False,
+        count_include_pad=None,
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if count_include_pad is not None:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+                count_include_pad=count_include_pad,
+            )
+        else:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.avg_pool3d))
+    run_and_verify(get_graph(relay.nn.max_pool3d))
+    run_and_verify(get_graph(relay.nn.max_pool3d, padding=(0, 0, 0, 1, 1, 1)))
+    run_and_verify(get_graph(relay.nn.max_pool3d, strides=(1, 1, 1)))
+
+
+def test_conv3d_transpose():
+    def get_graph(
+        x_shape=(1, 32, 8, 8, 8),
+        k_shape=(32, 16, 3, 3, 3),
+        groups=1,
+        padding=(0, 0, 0),
+        strides=(1, 1, 1),
+        output_padding=(0, 0, 0),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv3d_transpose(
+            x,
+            kernel,
+            channels=k_shape[1],
+            kernel_size=k_shape[2:5],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            output_padding=output_padding,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph(strides=(2, 2, 2)))
+    run_and_verify(get_graph(strides=(2, 2, 2), output_padding=(1, 1, 1)))
+
+
+def test_alexnet():
+    run_and_verify_model("alexnet")
+
+
+def test_resnet18_v1():
+    run_and_verify_model("resnet18_v1")
+
+
+def test_resnet18_v2():
+    run_and_verify_model("resnet18_v2")
+
+
+def test_squeezenet():
+    run_and_verify_model("squeezenet1.0")
+
+
+def test_mobilenet():
+    run_and_verify_model("mobilenet0.25")
+
+
+def test_mobilenet_v2():
+    run_and_verify_model("mobilenetv2_0.25")
+
+
+def test_vgg11():
+    run_and_verify_model("vgg11")
+
+
+def test_densenet121():
+    run_and_verify_model("densenet121")

Review comment:
       I don't think we need to test that many models in the CI. Maybe one ResNet and one MobileNet would be sufficient.

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):

Review comment:
       It's unclear what this function does especially you have `run_and_verify_model`. Maybe `run_and_verify_func`?

##########
File path: src/relay/backend/contrib/tensorrt/codegen.cc
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/tensorrt/codegen.cc
+ * \brief Implementation of the TensorRT JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../utils.h"
+#include "../codegen_json/codegen_json.h"
+
+#if TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#endif
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*! \brief Attributes to store the compiler options for TensorRT. */
+struct TensorRTCompilerConfigNode : public tvm::AttrsNode<TensorRTCompilerConfigNode> {
+  Array<Integer> tensorrt_version;
+  bool use_implicit_batch;
+  size_t max_workspace_size;
+  bool remove_no_mac_subgraphs;
+
+  TVM_DECLARE_ATTRS(TensorRTCompilerConfigNode, "ext.attrs.TensorRTCompilerConfigNode") {
+    TVM_ATTR_FIELD(tensorrt_version)
+        .describe("TensorRT version as (major, minor, patch).")
+        .set_default(Array<Integer>({6, 0, 1}));
+    TVM_ATTR_FIELD(use_implicit_batch).set_default(true);
+    TVM_ATTR_FIELD(max_workspace_size).set_default(size_t(1) << 30);
+    TVM_ATTR_FIELD(remove_no_mac_subgraphs).set_default(false);
+  }
+};
+
+class TensorRTCompilerConfig : public Attrs {
+ public:
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TensorRTCompilerConfig, Attrs,
+                                            TensorRTCompilerConfigNode);
+};
+
+TVM_REGISTER_NODE_TYPE(TensorRTCompilerConfigNode);
+TVM_REGISTER_PASS_CONFIG_OPTION("relay.ext.tensorrt.options", TensorRTCompilerConfig);
+
+/*!
+ * \brief Generates an TensorRTModule from a relay expression by serializing the expression to a
+ * json representation. TensorRT is not required here because use of TensorRT APIs is deferred until
+ * runtime.
+ */
+class TensorRTJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  TensorRTJSONSerializer(const std::string& symbol, const Expr& expr)
+      : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+
+    std::vector<JSONGraphNodeEntry> inputs;
+    for (const auto& arg : cn->args) {
+      auto res = VisitExpr(arg);
+      inputs.insert(inputs.end(), res.begin(), res.end());
+    }
+    auto node = std::make_shared<JSONGraphNode>(name,     /* name_ */
+                                                "kernel", /* op_type_ */
+                                                inputs, 1 /* num_outputs_ */);
+    if (name == "nn.pad") {
+      SetPadNodeAttribute(node, cn);
+    } else if (name == "strided_slice") {
+      SetStridedSliceNodeAttribute(node, cn);
+    } else {
+      SetCallNodeAttribute(node, cn);
+    }
+    // These attributes are global to the whole module.
+    SaveGlobalAttributes(node);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  void SetPadNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* pad_attr = cn->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    const int dim_h = (p.size() == 5) ? 3 : 2;
+    const int dim_w = (p.size() == 5) ? 4 : 3;
+    std::vector<std::string> padding = {std::to_string(p[dim_h][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_h][1].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    node->SetAttr("padding", padding_attr);
+  }
+
+  void SetStridedSliceNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* attrs = cn->attrs.as<StridedSliceAttrs>();
+    CHECK(attrs);
+    CHECK(attrs->begin && attrs->end && attrs->strides);

Review comment:
       We can merge these two checks and provide a proper error message.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_utils.h
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/utils.h
+ * \brief Helper functions used by TensorRTBuilder or TrtOpConverters.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+
+#include <string>
+#include <vector>
+
+#include "NvInfer.h"
+
+// There is a conflict between cpplint and clang-format-10.
+// clang-format off
+#define TRT_VERSION_GE(major, minor, patch)                                                    \
+  ((NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
+  (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
+// clang-format on
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+/*!
+ * \brief Helper function to convert an vector to TRT Dims.
+ * \param vec Vector.
+ * \return TRT Dims.
+ */
+template <typename T>
+inline nvinfer1::Dims VectorToTrtDims(const std::vector<T>& vec) {
+  nvinfer1::Dims dims;
+  // Dims(nbDims=0, d[0]=1) is used to represent a scalar in TRT.
+  dims.d[0] = 1;
+  dims.nbDims = vec.size();
+  for (size_t i = 0; i < vec.size(); ++i) {
+    dims.d[i] = vec[i];
+  }
+  return dims;
+}
+
+/*!
+ * \brief Helper function to convert TRT Dims to vector.
+ * \param vec TRT Dims.
+ * \return Vector.
+ */
+inline std::vector<int> TrtDimsToVector(const nvinfer1::Dims& dims) {
+  return std::vector<int>(dims.d, dims.d + dims.nbDims);
+}
+
+/*!
+ * \brief Helper function to convert vector to string.
+ * \param vec Vector.
+ * \return Vector as a string.
+ */
+template <typename T>
+inline std::string DebugString(const std::vector<T>& vec) {

Review comment:
       @zhiics do we have existing helper functions to achieve the same goal?

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):
+    if skip_codegen_test():
+        return
+    f, input_shapes, is_param = config
+    params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+    input_dict = {
+        k: np.random.uniform(-1, 1, v).astype(np.float32)
+        for k, v in input_shapes.items()
+        if k not in is_param
+    }
+
+    # Run TRT
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    # Run reference
+    mod = tvm.IRModule()
+    mod["main"] = f
+    with tvm.transform.PassContext(opt_level=3):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    assert len(results) == len(ref_results)
+    for i in range(len(results)):
+        res = results[i].asnumpy()
+        ref_res = ref_results[i].asnumpy()
+        assert res.shape == ref_res.shape
+        tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+
+def run_and_verify_model(model):
+    if skip_codegen_test():
+        return
+
+    def compile_and_run(i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+
+        def check_trt_used(graph):
+            import json
+
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum(
+                [
+                    1
+                    for n in graph["nodes"]
+                    if n.get("attrs", {}).get("func_name", "").startswith("tensorrt_")
+                ]
+            )
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={"data": input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod, config = tensorrt.partition_for_tensorrt(mod, params)
+            with tvm.transform.PassContext(
+                opt_level=3, config={"relay.ext.tensorrt.options": config}
+            ):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+            check_trt_used(graph)
+        else:
+            with tvm.transform.PassContext(opt_level=3):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+
+        if skip_runtime_test():
+            return
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return res
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    res = compile_and_run(i_data, input_shape, dtype, use_trt=True)
+    ref_res = compile_and_run(i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+    tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)
+
+
+def test_tensorrt_simple():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3, 1, 1)
+    zshape = (1, 1, 1, 1)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.var("y", shape=(yshape), dtype=dtype)
+    z = relay.var("z", shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_not_compatible():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 32, 14, 14)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_serialize():
+    if skip_codegen_test():
+        return
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    block = get_model("resnet18_v1", pretrained=True)
+    mod, params = relay.frontend.from_mxnet(
+        block, shape={"data": (1, 3, 224, 224)}, dtype="float32"
+    )
+    # Compile
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        lib = relay.build(mod, "cuda", params=params)
+    # Serialize
+    lib.export_library("compiled.so")
+    # Deserialize
+    loaded_lib = tvm.runtime.load_module("compiled.so")
+    # Run
+    if skip_runtime_test():
+        return
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib["default"](tvm.gpu(0)))
+    i_data = np.random.uniform(0, 1, (1, 3, 224, 224)).astype("float32")
+    for i in range(10):
+        gen_module.run(data=i_data)

Review comment:
       Why you need to run 10 times?

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {

Review comment:
       Could you remind me what other possible types here?

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       This is an interesting discussion. I realized that this is more like a serialization for platform-dependent TensorRT engines. If it's not possible to build and serialize the engine during the compilation (or cross-compilation) even we have built the TVM with TensorRT runtime, then this is probably inevitable; otherwise we may build the engine and serialize the bit-stream along with other artifacts in `SaveToBinary`.
   
   If the serialization here is inevitable, which I believe in it because users may not have TensorRT during compilation, then the next question is whether we can update the ".so" file with the serialized engine here instead of creating a separate file. In other words, the .so file may or may not contain a serialized engine, but if it has, we don't need to build it again.
   
   
   
   




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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707919898


   > I agree that we can merge it first. But before that, @trevor-m could you rebase against the master and run the tests again locally to see if all of them pass? I am not sure if everything is oaky after the diagnostic error reporting was merged.
   
   Thanks Zhi, I ran the `test_tensorrt.py` locally for both codegen only and with runtime and all of the tests passed.


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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r489579130



##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):

Review comment:
       This looks the same to `relay.bind`

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):
+        """
+        Visit an expression while replacing vars according to var_map. Used by
+        SubgraphRemover/PruneSubgraphs to return a subgraph originally partitioned to TRT back to TVM.
+        """
+        def __init__(self, var_map):
+            ExprMutator.__init__(self)
+            self.var_map = var_map
+
+        def visit_var(self, var):
+            if var in self.var_map:
+                return self.var_map[var]
+            return super().visit_var(var)
+
+    class SubgraphRemover(ExprMutator):

Review comment:
       Can we just remove the attributes of the function, e.g. inline, and then run the inline pass? 

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {
+    std::string cache_dir = dmlc::GetEnv("TVM_TENSORRT_CACHE_DIR", std::string(""));
+    if (cache_dir.empty()) return false;
+    std::string key = GetSubgraphKey();
+    std::string path = cache_dir + "/" + key + ".plan";
+    // Check if engine is in the cache.
+    std::ifstream infile(path, std::ios::binary);
+    if (!infile.good()) return false;
+    LOG(INFO) << "Loading cached TensorRT engine from " << path;

Review comment:
       I suggest we use DLOG for all

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       I am not sure if we need these two serialization methods. Can we just rely on LoadFrom/SaveToBinary?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \

Review comment:
       `if len(input_batch_sizes) > 1 and len(set(input_batch_sizes)) == 1`?




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

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



[GitHub] [incubator-tvm] trevor-m edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-701036415


   It looks like `/tests/scripts/task_cpp_unittest.sh` is failing at "Test MISRA-C runtime" in the CI:
   ```
   python3 build_model.py -o build --test
   INFO:compile_engine:Using injective.cpu for add based on highest priority (10)
   INFO:compile_engine:Using injective.cpu for add based on highest priority (10)
   make: *** [build/test_graph_c.json] Segmentation fault (core dumped)
   ```
   
   I wasn't able to reproduce it locally. I don't think my changes should affect this test - @areusch any ideas?


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-696860539


   > Finished reviewing the C++ part and the tests. I am not familiar with the TRT APIs so I didn't review the op converter in details.
   > 
   > For tests, how long does it need to run all the tests? I'm afraid that running all unit tests here will slow the CI a lot.
   
   Thanks Cody, I measured the time on a AWS g4dn.4xlarge instance.
   
   Full with runtime: 6m2.021s
   Codegen only: 1m0.358s
   Codegen only, removed all integration test models except resnet18_v1 and mobilenet_v2: 0m51.536s
   
   Seems like the extra models in the integration tests do not consume much time.


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r491051930



##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       AFAIK `SaveToBinary` is only ever invoked during compilation.
   
   The engine is only built during runtime because it is specific to the target GPU and platform, so `CacheEnginesToDisk` needs to be performed by the runtime also.
   
   See https://docs.nvidia.com/deeplearning/tensorrt/developer-guide/index.html#work
   
   To op
   
   > timize your model for inference, TensorRT takes your network definition, performs optimizations including platform-specific optimizations, and generates the inference engine. This process is referred to as the build phase. The build phase can take considerable time, especially when running on embedded platforms. Therefore, a typical application will build an engine once, and then serialize it as a plan file for later use.
   > 
   > Note: The generated plan files are not portable across platforms or TensorRT versions. Plans are specific to the exact GPU model they were built on (in addition to the platforms and the TensorRT version) and must be re-targeted to the specific GPU in case you want to run them on a different GPU.
   

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       AFAIK `SaveToBinary` is only ever invoked during compilation.
   
   The engine is only built during runtime because it is specific to the target GPU and platform, so `CacheEnginesToDisk` needs to be performed by the runtime also.
   
   See https://docs.nvidia.com/deeplearning/tensorrt/developer-guide/index.html#work
   
   
   
   > To optimize your model for inference, TensorRT takes your network definition, performs optimizations including platform-specific optimizations, and generates the inference engine. This process is referred to as the build phase. The build phase can take considerable time, especially when running on embedded platforms. Therefore, a typical application will build an engine once, and then serialize it as a plan file for later use.
   > 
   > Note: The generated plan files are not portable across platforms or TensorRT versions. Plans are specific to the exact GPU model they were built on (in addition to the platforms and the TensorRT version) and must be re-targeted to the specific GPU in case you want to run them on a different GPU.
   




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       The message is a bit confusing. `USE_TENSORRT` means enabling the TensorRT codegen for graph partitininog. It doesn't require TensorRT to be available in the system environment. IIUC, maybe it's better to say "Build with TensorRT codegen", although I just found that "Build with Arm Compute Library" has the same issue.
   
   @lhutton1 could you also share your thoughts for this?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch

Review comment:
       Do we need a guard here as in L52?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):
+        """
+        Visit an expression while replacing vars according to var_map. Used by
+        SubgraphRemover/PruneSubgraphs to return a subgraph originally partitioned to TRT back to TVM.
+        """
+        def __init__(self, var_map):
+            ExprMutator.__init__(self)
+            self.var_map = var_map
+
+        def visit_var(self, var):
+            if var in self.var_map:
+                return self.var_map[var]
+            return super().visit_var(var)
+
+    class SubgraphRemover(ExprMutator):

Review comment:
       This is the issue we discussed in this PR about how to deal with post-partitioning judgements. We could later on figure out an approach to generalize this requirement.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]

Review comment:
       ```suggestion
       ret: Tuple[int, int, int]
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]

Review comment:
       ```suggestion
       version : Optional[Tuple[int, int, int]]
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.

Review comment:
       Shoud be fine in one line.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs

Review comment:
       Do we need a guard here as in L52?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper

Review comment:
       These two functions share the same logic so it would be better to merge them:
   
   ```python
   def _register_external_op_helper_with_checker(op_name, checker):
       # The body of _register_external_op_helper_func
   
   def _register_external_op_helper(op_name, supported=True):
       return _register_external_op_helper_with_checker(op_name, lambda attrs, args, op_name: supported)
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.

Review comment:
       ```suggestion
       mod_n_config: Tuple[Module, Dict[str, Any]]
           A tuple of 1) annotated and partitioned module and 2) "relay.ext.tensorrt.options" configuration
           which should be given to PassContext when building.
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")

Review comment:
       - Use the standard logger: `logger = logging.getLogger("TensorRT")`.
   - "info" seems not a right log level for messages like this. Should be "warn" or "debug".
   - Ditto to other logs.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.

Review comment:
       s/relay/Relay/g

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================

Review comment:
       ```suggestion
   ==========================
   ```

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.

Review comment:
       Better to provide an overview of this tutorial, including what information/takeaway readers could expect after reading this article.
   
   After reading through, I would suggest mentioning the following points:
   1. (optional) Some model performance numbers to impressive readers.
   2. Install TensorRT and build TVM with it.
   3. Build a ResNet-18 with TensorRT.
   4. Operator support (I also suggest moving this section to the end along with the last section about adding new operators.)
   5. Deploy the built model with TensorRT runtime.
   6. How to support other operators.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------

Review comment:
       ```suggestion
   -------------------
   ```
   Note that misaligning will cause doc generation warnings and failed the CI.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.

Review comment:
       Better to say this section is tested with TensorRT version XXX and may not stay up-to-date with newer versions that changes the APIs.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (
+        not get_tensorrt_use_implicit_batch_mode()
+        and (isinstance(args[0], Constant) or isinstance(args[1], Constant))
+        and args[0].checked_type.shape[0] == args[1].checked_type.shape[0]
+        and args[0].checked_type.shape[0] != 1
+        and (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3)
+    ):
+        logging.info("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.batch_norm is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        logging.info("nn.batch_norm: axis is %d but must be 1 or 3.", int(attrs.axis))
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.softmax is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info("nn.conv2d: kernel_layout is %s but must be OIHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if dense is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.dense: input has rank %d but must be 2, 3 or 4.", input_rank)
+        return False
+    if weight_rank != 2:
+        logging.info("nn.dense: weight has rank %d but must be 2.", weight_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.bias_add is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.bias_add: input rank is %d but must be 2, 3 or 4.", input_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.avg_pool2d: layout is %d but must be NCHW.", attrs.layout)
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        logging.info(
+            "nn.avg_pool2d: inclusive-counted blended or average "
+            "pooling is not supported in combination with asymmetric padding"
+        )
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_avg_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if expand_dims is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if squeeze is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        logging.info("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        logging.info("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if concatenate is supported by TensorRT."""
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        logging.info("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                logging.info("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info(
+            "nn.conv2d_transpose: kernel_layout is %s but must be OIHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        logging.info("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def layout_transform_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if layout_transform is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [
+        ("NCHW", "NHWC"),
+        ("NHWC", "NCHW"),
+        ("NDHWC", "NCDHW"),
+        ("NCDHW", "NDHWC"),
+    ]:
+        logging.info(
+            "layout_transform: %s to %s is not supported.", attrs.src_layout, attrs.dst_layout
+        )
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if reshape is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        logging.info("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            logging.info("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            logging.info("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.pad is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        logging.info("nn.pad: pad mode is %s but must be constant.", attrs.pad_mode)
+        return False
+    if float(attrs.pad_value) != 0.0:
+        logging.info("nn.pad: pad value is %f but must be 0.0.", float(attrs.pad_value))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        logging.info("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        logging.info("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+
+def reduce_annotate_fn(attrs, args, op_name):
+    """Helper for reduce operations."""
+    if not attrs.axis or len(attrs.axis) == 0:
+        logging.info("%s: cannot reduce to scalar.", op_name)
+        return False
+    if attrs.exclude:
+        logging.info("%s: exclude not supported.", op_name)
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        logging.info("%s: can't modify batch dimension.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    """Helper for ops which require TRT 5.1.5 or greater."""
+    if get_tensorrt_version() < (5, 1, 5):
+        logging.info("%s: requires TensorRT version 5.1.5 or higher.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if strided_slice is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = (
+            attrs.end[0] is not None
+            and int(attrs.end[0]) != -1
+            and int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        )
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            logging.info("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        logging.info("strided_slice: stride must be positive")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):

Review comment:
       It might be better to generalize `trt_5_1_5_annotate_fn` to `trt_version_annotate_fn(version, attrs, args, op_name)`.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be "cuda".

Review comment:
       This statement is not 100% correct. It can also be "cuda -model=v100" for example.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage

Review comment:
       Explicitly say what you are doing in this section, such as "Build and Deploy ResNet-18 with TensorRT"

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (
+        not get_tensorrt_use_implicit_batch_mode()
+        and (isinstance(args[0], Constant) or isinstance(args[1], Constant))
+        and args[0].checked_type.shape[0] == args[1].checked_type.shape[0]
+        and args[0].checked_type.shape[0] != 1
+        and (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3)
+    ):
+        logging.info("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.batch_norm is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        logging.info("nn.batch_norm: axis is %d but must be 1 or 3.", int(attrs.axis))
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.softmax is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info("nn.conv2d: kernel_layout is %s but must be OIHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if dense is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.dense: input has rank %d but must be 2, 3 or 4.", input_rank)
+        return False
+    if weight_rank != 2:
+        logging.info("nn.dense: weight has rank %d but must be 2.", weight_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.bias_add is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.bias_add: input rank is %d but must be 2, 3 or 4.", input_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.avg_pool2d: layout is %d but must be NCHW.", attrs.layout)
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        logging.info(
+            "nn.avg_pool2d: inclusive-counted blended or average "
+            "pooling is not supported in combination with asymmetric padding"
+        )
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_avg_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if expand_dims is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if squeeze is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        logging.info("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        logging.info("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if concatenate is supported by TensorRT."""
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        logging.info("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                logging.info("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info(
+            "nn.conv2d_transpose: kernel_layout is %s but must be OIHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        logging.info("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def layout_transform_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if layout_transform is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [
+        ("NCHW", "NHWC"),
+        ("NHWC", "NCHW"),
+        ("NDHWC", "NCDHW"),
+        ("NCDHW", "NDHWC"),
+    ]:
+        logging.info(
+            "layout_transform: %s to %s is not supported.", attrs.src_layout, attrs.dst_layout
+        )
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if reshape is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        logging.info("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            logging.info("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            logging.info("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.pad is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        logging.info("nn.pad: pad mode is %s but must be constant.", attrs.pad_mode)
+        return False
+    if float(attrs.pad_value) != 0.0:
+        logging.info("nn.pad: pad value is %f but must be 0.0.", float(attrs.pad_value))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        logging.info("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        logging.info("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+
+def reduce_annotate_fn(attrs, args, op_name):
+    """Helper for reduce operations."""
+    if not attrs.axis or len(attrs.axis) == 0:
+        logging.info("%s: cannot reduce to scalar.", op_name)
+        return False
+    if attrs.exclude:
+        logging.info("%s: exclude not supported.", op_name)
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        logging.info("%s: can't modify batch dimension.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    """Helper for ops which require TRT 5.1.5 or greater."""
+    if get_tensorrt_version() < (5, 1, 5):
+        logging.info("%s: requires TensorRT version 5.1.5 or higher.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if strided_slice is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = (
+            attrs.end[0] is not None
+            and int(attrs.end[0]) != -1
+            and int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        )
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            logging.info("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        logging.info("strided_slice: stride must be positive")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        logging.info("nn.conv3d: data_layout is %s but must be NCDHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        logging.info("nn.conv3d: kernel_layout is %s but must be OIDHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        logging.info("nn.conv3d: out_layout is %s but must be NCDHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        logging.info("nn.max_pool3d: layout is %s but must be NCDHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        logging.info("nn.avg_pool3d: layout is %s but must be NCDHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        logging.info("nn.conv3d_transpose: data_layout is %s but must be NCDHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        logging.info(
+            "nn.conv3d_transpose: kernel_layout is %s but must be OIDHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        logging.info("nn.conv3d_transpose: out_layout is %s but must be NCDHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        logging.info("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+
+def is_valid_subgraph(params, body):
+    """Final check on whether the subgraph is valid and should be offloaded to TensorRT."""
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        logging.info("tensorrt: scalar inputs not supported")
+                        return False
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    logging.info("tensorrt: scalar inputs not supported")
+                    return False
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and len(set(input_batch_sizes)) != 1:
+            logging.info("tensorrt: inputs have different batch sizes")
+            return False
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return False
+    return True
+
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):

Review comment:
       As it is already prune_"tensorrt"_subgraphs, it looks to me that you don't need to specify `target` in the arguments?




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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r492407348



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       Thanks for the review Cody!
   
   You're right, the names aren't really that clear here. Originally, I had them as `USE_TENSORRT_CODEGEN` for codegen only and `USE_TENSORRT` for both codegen and runtime. I changed them to match the ACL definitions.




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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-693018923


   There appears to be an inconsistency in the CI between `tests/lint/cpplint.sh` and `tests/lint/clang_format.sh` for this macro definition:
   ```
   #define TRT_VERSION_GE(major, minor, patch)                                                    \
     ((NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
      (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
   ```
   
   `tests/lint/cpplint.sh` will complain about the 3 spaces indent on the 3rd line:
   ```
   src/runtime/contrib/tensorrt/tensorrt_utils.h:35:  Weird number of spaces at line-start.  Are you using a 2-space indent?  [whitespace/indent] [3]
   ```
   
   If I change it to 2 spaces, the cpplint passes but the clang-format checker fails:
   ```
   ---------clang-format log----------
   diff --git a/src/runtime/contrib/tensorrt/tensorrt_utils.h b/src/runtime/contrib/tensorrt/tensorrt_utils.h
   index 6d664e47d..746726fc1 100644
   --- a/src/runtime/contrib/tensorrt/tensorrt_utils.h
   +++ b/src/runtime/contrib/tensorrt/tensorrt_utils.h
   @@ -32,7 +32,7 @@
   
    #define TRT_VERSION_GE(major, minor, patch)                                                    \
      ((NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
   -  (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
   +   (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
   
    namespace tvm {
    namespace runtime {
   
   clang-format lint error found. Consider running clang-format-10 on these files to fix them.
   script returned exit code 1
   ```
   
   Is there a reason we have both of these checks? They appear to do the same thing but want different things. Is there a way to override one of them?


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r489717587



##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):
+        """
+        Visit an expression while replacing vars according to var_map. Used by
+        SubgraphRemover/PruneSubgraphs to return a subgraph originally partitioned to TRT back to TVM.
+        """
+        def __init__(self, var_map):
+            ExprMutator.__init__(self)
+            self.var_map = var_map
+
+        def visit_var(self, var):
+            if var in self.var_map:
+                return self.var_map[var]
+            return super().visit_var(var)
+
+    class SubgraphRemover(ExprMutator):

Review comment:
       I originally tried that approach. However, when the tensorrt subgraphs are inlined, TVM will try to optimize the code in the tensorrt subgraphs (for example it will change conv2d to contrib_conv2d_winograd_without_weight_transform) which we don't want.




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

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



[GitHub] [incubator-tvm] trevor-m edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707363920


   > @trevor-m I don't see anything weird with your build rules, but I wonder if changing the cmake config affected something. do you have >1 CI failure showing the segfault, or can you reproduce this locally?
   > 
   > we have seen this before sporadically but don't know what causes it, and it's usually pretty hard to reproduce
   
   Yes, every CI run with USE_TENSORRT_CODEGEN ON got the segfault (there was at least 10 runs).
   I was able to reproduce this consistently locally now by replicating the same steps used by the CI (using docker image). It's the `apps/bundle_deploy/build_model.py` script which is segfaulting. I ran it using gdb inside the container.
   
   From `gdb -ex r --args python3 build_model.py -o build --test`.
   ```
   Thread 1 "python3" received signal SIGSEGV, Segmentation fault.
   __GI___libc_free (mem=0x6) at malloc.c:2958
   2958    malloc.c: No such file or directory.
   (gdb) bt
   #0  __GI___libc_free (mem=0x6) at malloc.c:2958
   #1  0x00007fffde4937f4 in dmlc::parameter::FieldAccessEntry::~FieldAccessEntry() () from /workspace/build/libtvm.so
   #2  0x00007fff9702a4af in dmlc::parameter::FieldEntry<std::string>::~FieldEntry() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #3  0x00007fff97037267 in dmlc::parameter::ParamManager::~ParamManager() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #4  0x00007ffff6cd7008 in __run_exit_handlers (status=0, listp=0x7ffff70615f8 <__exit_funcs>, run_list_atexit=run_list_atexit@entry=true) at exit.c:82
   #5  0x00007ffff6cd7055 in __GI_exit (status=<optimized out>) at exit.c:104
   #6  0x00007ffff6cbd847 in __libc_start_main (main=0x4d1cb0 <main>, argc=5, argv=0x7fffffffe858, init=<optimized out>, fini=<optimized out>, rtld_fini=<optimized out>, stack_end=0x7fffffffe848) at ../csu/libc-start.c:325
   #7  0x00000000005e8569 in _start ()
   ```
   
   I found that the issue is caused by the xgboost version in the CI docker is too old. It had 1.0.2. I upgraded to 1.2 and now the segfault is fixed.


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

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



[GitHub] [incubator-tvm] trevor-m edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707363920


   > @trevor-m I don't see anything weird with your build rules, but I wonder if changing the cmake config affected something. do you have >1 CI failure showing the segfault, or can you reproduce this locally?
   > 
   > we have seen this before sporadically but don't know what causes it, and it's usually pretty hard to reproduce
   
   Yes, every CI run with USE_TENSORRT_CODEGEN ON got the segfault (there was at least 10 runs).
   I was able to reproduce this consistently locally now by replicating the same steps used by the CI (using docker image). It's the `apps/bundle_deploy/build_model.py` script which is segfaulting. I ran it using gdb inside the container.
   
   From `gdb -ex r --args python3 build_model.py -o build --test`.
   ```
   Thread 1 "python3" received signal SIGSEGV, Segmentation fault.
   __GI___libc_free (mem=0x6) at malloc.c:2958
   2958    malloc.c: No such file or directory.
   (gdb) bt
   #0  __GI___libc_free (mem=0x6) at malloc.c:2958
   #1  0x00007fffde4937f4 in dmlc::parameter::FieldAccessEntry::~FieldAccessEntry() () from /workspace/build/libtvm.so
   #2  0x00007fff9702a4af in dmlc::parameter::FieldEntry<std::string>::~FieldEntry() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #3  0x00007fff97037267 in dmlc::parameter::ParamManager::~ParamManager() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #4  0x00007ffff6cd7008 in __run_exit_handlers (status=0, listp=0x7ffff70615f8 <__exit_funcs>, run_list_atexit=run_list_atexit@entry=true) at exit.c:82
   #5  0x00007ffff6cd7055 in __GI_exit (status=<optimized out>) at exit.c:104
   #6  0x00007ffff6cbd847 in __libc_start_main (main=0x4d1cb0 <main>, argc=5, argv=0x7fffffffe858, init=<optimized out>, fini=<optimized out>, rtld_fini=<optimized out>, stack_end=0x7fffffffe848) at ../csu/libc-start.c:325
   #7  0x00000000005e8569 in _start ()
   ```
   
   I found that the issue is caused by the xgboost version in the CI docker is too old. It had 1.0.2. I upgrade to 1.2 and now the segfault is fixed.


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692328667


   > > Hmm, this seems like it would make the job of the `PruneTensorRTSubgraph` pass much more difficult. `PartitionGraph` already takes care of collecting the inputs and outputs of a subgraph and additional processing such as making sure there are no duplicate outputs. If `PruneTesnorRTCompilerRegion` was before `PartitionGraph`, it would have to duplicate a lot of that work. The idea of the pruning pass is that we should present each backend with the final subgraph exactly as it would be when it is passed to the codegen and the backend should decide if it is valid or not. Are you concerned about the overhead of partitioning a subgraph which would be later discarded?
   > > Btw just for referece, here is the general implementation of PruneSubgraph that I originally implemented: [trevor-m@06015a4](https://github.com/trevor-m/tvm/commit/06015a4617cfaad56adcaa0c71b485d6bd711128)
   > 
   > My main concern was that it would be tedious to have a `partition_graph -> revert_some_partitions` flow. Also in this case, your post-processing pass depends on the partition pass and may fail along with the change of the partition pass. If this requirement is important, I'd even prefer to add post-processing feature to the partition pass that allows you to provide a packed function to check if a partitioned function is valid.
   > 
   > On the other hand, in order to not block this PR for too long, we can maybe follow the current flow first, and discuss a plan of refactoring the partition pass to better support this requirement.
   > 
   > @zhiics do you have any suggestion?
   
   Yeah, I think its okay to have a refinement pass for TRT ATM since doing such a decision in the current partitioning is not easy. In the long run, we should make the partitioning pass more intelligent by taking in some configurations and partitioning over the region accordingly. Or we can consider some of the configs when merging the regions. That would need more investigation.
   
   
   


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686726858


   cc @comaniac @mbaret @lhutton1 @masahi @leandron @mbrookhart 


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-701059299


   > @trevor-m i agree this seems unrelated. can you try retriggering the CI just to double-check it's reproducible? there was an issue with flaky bundle_deploy tests a while back, but we thought we fixed that. what command are you running locally?
   
   Thanks @areusch 
   It's actually failed a few times already but I just triggered it again so we will see.
   I'm running this locally after TVM is finished building:
   ```
   cd apps/bundle_deploy
   rm -rf build
   make test_dynamic test_static
   ```


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

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



[GitHub] [incubator-tvm] zhiics edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707922105


   I think we can enable the test and merge after #6679 is landed since its pretty close already? Sorry for the back and forth.


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

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



[GitHub] [incubator-tvm] masahi commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
masahi commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686768783


   >     1. Currently, I'm using environment variables to pass these from python to the codegen in C++. I wonder if there is a better way to do this?
   
   How about using Config mechanism? I learned about this from ethos integration (thanks @mbaret) and it cleaned up my code as well. 
   
   https://github.com/apache/incubator-tvm/blob/30cd2302e4078b3a8787e30d70fd79e5b729ec82/src/relay/backend/contrib/ethosn/codegen_ethosn.h#L219


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686779607


   @trevor-m @masahi for the pass config, we may not be able to obtain at the runtime though


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-701036415


   It looks like `/tests/scripts/task_cpp_unittest.sh` is failing at "Test MISRA-C runtime":
   ```
   python3 build_model.py -o build --test
   INFO:compile_engine:Using injective.cpu for add based on highest priority (10)
   INFO:compile_engine:Using injective.cpu for add based on highest priority (10)
   make: *** [build/test_graph_c.json] Segmentation fault (core dumped)
   ```
   
   I wasn't able to reproduce it locally. I don't think my changes should affect this test - @areusch any ideas?


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

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



[GitHub] [incubator-tvm] lhutton1 commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
lhutton1 commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r488545828



##########
File path: cmake/modules/contrib/TensorRT.cmake
##########
@@ -0,0 +1,55 @@
+# 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.
+
+# TensorRT Codegen only. This can be enabled independently of USE_TENSORRT to

Review comment:
       ```suggestion
   # TensorRT Codegen only. This can be enabled independently of USE_TENSORRT_GRAPH_RUNTIME to
   ```

##########
File path: src/runtime/contrib/tensorrt/tensorrt_builder.cc
##########
@@ -0,0 +1,224 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_builder.cc
+ * \brief The TensorRTBuilder class can be used to convert a JSONRuntime graph into a TRT engine
+ * which can be used for inference.
+ */
+
+#include "tensorrt_builder.h"
+
+#include <tvm/runtime/ndarray.h>
+
+#include <memory>
+#include <string>
+
+#include "tensorrt_logger.h"
+#include "tensorrt_ops.h"
+#include "tensorrt_utils.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+TensorRTBuilder::TensorRTBuilder(TensorRTLogger* logger, size_t max_workspace_size,
+                                 bool use_implicit_batch, bool use_fp16, int batch_size)
+    : max_workspace_size_(max_workspace_size),
+      use_implicit_batch_(use_implicit_batch),
+      use_fp16_(use_fp16),
+      batch_size_(batch_size) {
+  // Create TRT builder and network.
+  builder_ = nvinfer1::createInferBuilder(*logger);
+#if TRT_VERSION_GE(6, 0, 1)
+  // Use INetworkV2.
+  auto flags =
+      1U << static_cast<uint32_t>(nvinfer1::NetworkDefinitionCreationFlag::kEXPLICIT_BATCH);
+  if (use_implicit_batch_) {
+    flags = 0U;
+    builder_->setMaxBatchSize(batch_size_);
+  }
+  network_ = builder_->createNetworkV2(flags);
+#else
+  // Use INetwork with implicit batch.
+  builder_->setMaxBatchSize(batch_size_);
+  builder_->setMaxWorkspaceSize(max_workspace_size_);
+  builder_->setFp16Mode(use_fp16_);
+  network_ = builder_->createNetwork();
+#endif
+}
+
+void TensorRTBuilder::AddInput(int nid, const JSONGraphNode& node) {
+  auto node_name = node.GetOpName();
+  auto shapes = node.GetOpShape();
+  auto dtypes = node.GetOpDataType();
+  CHECK_EQ(shapes.size(), dtypes.size());
+  node_output_map_[nid] = {};
+  for (size_t i = 0; i < shapes.size(); ++i) {
+    const std::string name = node_name + "_" + std::to_string(i);
+    auto shape = shapes[i];
+    // Remove batch dim when not in explicit batch mode.
+    if (use_implicit_batch_ && shape.size() > 1) {
+      shape.erase(shape.begin());
+    }
+    DLOG(INFO) << "TRT input: " << name << " " << DebugString(shape);

Review comment:
       Is it useful to log every input and output? I feel like it might be a bit unnecessary 

##########
File path: src/runtime/contrib/tensorrt/tensorrt_builder.h
##########
@@ -0,0 +1,156 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_builder.h
+ * \brief The TensorRTBuilder class can be used to convert a JSONRuntime graph into a TRT engine
+ * which can be used for inference.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_logger.h"
+#include "tensorrt_ops.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+/*!
+ * \brief The product of TensorRTBuilder which provides everything needed to
+ * perform inference.
+ */
+struct TrtEngineAndContext {
+  nvinfer1::ICudaEngine* engine;
+  nvinfer1::IExecutionContext* context;
+  std::vector<std::string> inputs;
+  std::vector<std::string> outputs;
+};
+
+/*!
+ * \brief Converts a JSONRuntime graph into a TensorRT engine and execution context. Inputs,
+ * constants, layers, and outputs can be added to construct the TensorRT network definition. BuildEngine will then 

Review comment:
       Missing last part of sentence?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,675 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+TENSORRT_VERSION = (6, 0, 1)
+USE_IMPLICIT_BATCH = True
+REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    linked_ver = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+    if len(linked_ver) == 3:
+        return linked_ver
+    return TENSORRT_VERSION
+
+def set_tensorrt_version(version):
+    """Override TensorRT version for annotation
+
+    Returns
+    -------
+    version: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, an empty tuple is returned instead.
+    """
+    global TENSORRT_VERSION
+    TENSORRT_VERSION = version
+
+def get_tensorrt_use_implicit_batch_mode():
+    return USE_IMPLICIT_BATCH
+
+def set_tensorrt_use_implicit_batch_mode(use_implicit_batch):
+    global USE_IMPLICIT_BATCH
+    USE_IMPLICIT_BATCH = use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    return REMOVE_NO_MAC_SUBGRAPHS
+
+def set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs):
+    global REMOVE_NO_MAC_SUBGRAPHS
+    REMOVE_NO_MAC_SUBGRAPHS = remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True, remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). Will use linked TRT version if available if version is not specified.
+    use_implicit_batch : Optional[bool]
+
+    remove_no_mac_subgraphs : Optional[bool]
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        set_tensorrt_version(version)
+    set_tensorrt_use_implicit_batch_mode(use_implicit_batch)
+    set_tensorrt_remove_no_mac_subgraphs(remove_no_mac_subgraphs)
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3):
+        mod = seq(mod)
+    mod = prune_tensorrt_subgraphs(mod)
+    # Pass parameters to codegen
+    os.environ["TVM_TENSORRT_USE_IMPLICIT_BATCH"] = str(int(use_implicit_batch))
+    os.environ["TVM_TENSORRT_MAX_WORKSPACE_SIZE"] = str(int(max_workspace_size))
+    return mod
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[0].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) != 1 and int(attrs.axis) != 3:
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank < 2 or input_rank > 4:
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (5, 1, 5):
+        print("strided_slice: requires TensorRT version 5.1.5 or higher.")
+        return False

Review comment:
       ```suggestion
       if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
           return False
   ```

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,573 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+def should_skip():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("skip because cuda is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("skip because tensorrt runtime is not available")
+        return True
+    return False
+
+def test_tensorrt_simple():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3,  1,  1)
+    zshape = (1,  1,  1,  1)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.var('y', shape=(yshape), dtype=dtype)
+    z = relay.var('z', shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_not_compatible():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 32, 14, 14)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_ops():
+    if should_skip():
+        return
+    def run_and_verify(config):
+        f, input_shapes, is_param = config
+        params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+        input_dict = {k: np.random.uniform(-1, 1, v).astype(np.float32) for k, v in input_shapes.items() if k not in is_param}
+
+        # Run TRT 
+        mod = tvm.IRModule()
+        mod['main'] = f
+        mod = tensorrt.partition_for_tensorrt(mod, params)
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**graph_params)
+        mod.run(**input_dict)
+        results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+        # Run reference
+        mod = tvm.IRModule()
+        mod['main'] = f
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**graph_params)
+        mod.run(**input_dict)
+        ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+        
+        assert len(results) == len(ref_results)
+        for i in range(len(results)):
+            res = results[i].asnumpy()
+            ref_res = ref_results[i].asnumpy()
+            assert res.shape == ref_res.shape
+            tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)

Review comment:
       Same as above

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,573 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+def should_skip():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("skip because cuda is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("skip because tensorrt runtime is not available")
+        return True
+    return False
+
+def test_tensorrt_simple():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3,  1,  1)
+    zshape = (1,  1,  1,  1)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.var('y', shape=(yshape), dtype=dtype)
+    z = relay.var('z', shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_not_compatible():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 32, 14, 14)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_ops():
+    if should_skip():
+        return
+    def run_and_verify(config):
+        f, input_shapes, is_param = config
+        params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+        input_dict = {k: np.random.uniform(-1, 1, v).astype(np.float32) for k, v in input_shapes.items() if k not in is_param}
+
+        # Run TRT 
+        mod = tvm.IRModule()
+        mod['main'] = f
+        mod = tensorrt.partition_for_tensorrt(mod, params)
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**graph_params)
+        mod.run(**input_dict)
+        results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+        # Run reference
+        mod = tvm.IRModule()
+        mod['main'] = f
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**graph_params)
+        mod.run(**input_dict)
+        ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+        
+        assert len(results) == len(ref_results)
+        for i in range(len(results)):
+            res = results[i].asnumpy()
+            ref_res = ref_results[i].asnumpy()
+            assert res.shape == ref_res.shape
+            tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+    def test_conv2d(x_shape=(1, 32, 8, 8), k_shape=(16, 32, 3, 3), groups=1, padding=(0, 0), strides=(1, 1), dilation=(1, 1)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        out = relay.nn.conv2d(x, kernel, channels=k_shape[0], kernel_size=k_shape[2:4], groups=groups, padding=padding, strides=strides, dilation=dilation)
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+    
+    def test_conv2d_nhwc(x_shape=(1, 8, 8, 32), k_shape=(3, 3, 32, 16)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        out = relay.nn.conv2d(x, kernel, channels=16, kernel_size=(3, 3), data_layout="NHWC", kernel_layout="HWIO")
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+
+    def test_conv2d_const_weights(x_shape=(1, 32, 8, 8), k_shape=(16, 32, 3, 3), groups=1, padding=(0, 0), strides=(1, 1), dilation=(1, 1)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.const(np.ones(k_shape).astype("float32"))
+        out = relay.nn.conv2d(x, kernel, channels=k_shape[0], kernel_size=k_shape[2:4], groups=groups, padding=padding, strides=strides, dilation=dilation)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+    
+    def test_dense(x_shape=(1, 16), k_shape=(32, 16)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        # Dense requires constant weights in TensorRT, so the weights are transposed by us.
+        out = relay.nn.dense(x, kernel, units=k_shape[0])
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+    
+    def test_bias_add(x_shape=(1, 16), channels=16):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        bias = relay.var('bias', shape=(channels,), dtype='float32')
+        out = relay.nn.bias_add(x, bias)
+        f = relay.Function([x, bias], out)
+        return f, {'x': x_shape, 'bias': (channels,)}, ['bias']
+    
+    def test_pool2d(op, x_shape=(1, 3, 32, 32), pool_size=(2, 2), strides=(2, 2), padding=(0, 0), ceil_mode=False, count_include_pad=None):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        if count_include_pad is not None:
+            out = op(x, pool_size=pool_size, strides=strides, padding=padding, ceil_mode=ceil_mode, count_include_pad=count_include_pad)
+        else:
+            out = op(x, pool_size=pool_size, strides=strides, padding=padding, ceil_mode=ceil_mode)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_global_pool2d(op, x_shape=(1, 3, 32, 32)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_batch_flatten(x_shape=(1, 3, 4, 6)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.nn.batch_flatten(x)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+    
+    def test_expand_dims(x_shape=(1, 3), axis=1, num_newaxis=1):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.expand_dims(x, axis, num_newaxis)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_squeeze(x_shape, axis):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.squeeze(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+    
+    def test_concatenate(input_shapes, axis):
+        concat_inputs = []
+        shapes_dict = {}
+        for i in range(len(input_shapes)):
+            name = 'input_{}'.format(i)
+            concat_inputs.append(relay.var(name, shape=(input_shapes[i]), dtype='float32'))
+            shapes_dict[name] = input_shapes[i]
+        out = relay.concatenate(concat_inputs, axis)
+        f = relay.Function(concat_inputs, out)
+        return f, shapes_dict, []
+    
+    def test_conv2d_transpose(x_shape=(1, 32, 8, 8), k_shape=(32, 16, 3, 3), groups=1, padding=(0, 0), strides=(1, 1)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        out = relay.nn.conv2d_transpose(x, kernel, channels=k_shape[1], kernel_size=k_shape[2:4], groups=groups, padding=padding, strides=strides)
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+    
+    def test_reshape(x_shape, new_shape):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.reshape(x, new_shape)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+    
+    def test_transpose(x_shape, order):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.transpose(x, order)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_transpose_weights_conv2d(x_shape=(1, 32, 9, 9), k_shape=(3, 3, 32, 16), order=(3, 2, 0, 1)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        kernel_t = relay.transpose(kernel, order)
+        # Conv2d requires constant weights in TensorRT, so the weights are transposed by us.
+        out = relay.nn.conv2d(x, kernel_t, channels=k_shape[order[0]], kernel_size=(3, 3))
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+
+    def test_transpose_weights_dense(x_shape=(1, 16), k_shape=(16, 32)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        kernel_t = relay.transpose(kernel, (1, 0))
+        # Dense requires constant weights in TensorRT, so the weights are transposed by us.
+        out = relay.nn.dense(x, kernel_t)
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+
+    def test_dense_from_pytorch(x_shape=(1, 16), k_shape=(32, 16)):
+        # FoldConstant will fold away the tranpose -> mult -> transpose.
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        kernel_t = relay.transpose(kernel, (1, 0))
+        beta = relay.const(1, dtype='float32')
+        kernel_t = relay.multiply(kernel_t, beta)
+        kernel_t = relay.transpose(kernel_t, (1, 0))
+        out = relay.nn.dense(x, kernel_t)
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+
+    def test_float_const(x_shape=(1, 16)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        beta = relay.const(1, dtype='float32')
+        out = relay.multiply(x, beta)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_pad(x_shape, pad_width):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.nn.pad(x, pad_width=pad_width)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_softmax(x_shape, axis):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.nn.softmax(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_batch_norm(x_shape, param_shape, axis=1, epsilon=1e-5):
+        x = relay.var("x", shape=(x_shape), dtype='float32')
+        beta = relay.var("beta", shape=(param_shape), dtype='float32')
+        gamma = relay.var("gamma",  shape=(param_shape), dtype='float32')
+        moving_mean = relay.var("moving_mean", shape=(param_shape), dtype='float32')
+        moving_var = relay.var("moving_var", shape=(param_shape), dtype='float32')
+        out, _, _ = relay.nn.batch_norm(x, gamma=gamma, beta=beta, moving_mean=moving_mean, moving_var=moving_var,
+                                        axis=axis, center=True, scale=True, epsilon=epsilon)
+        f = relay.Function([x, gamma, beta, moving_mean, moving_var], out)
+        return f, {'x': x_shape, 'beta': param_shape, 'gamma': param_shape,
+                   'moving_mean': param_shape, 'moving_var': param_shape}, ['beta', 'gamma', 'moving_mean', 'moving_var']
+
+    def test_unary(op, x_shape=(1, 8, 3, 3)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_clip(x_shape=(1, 8, 3, 3)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.clip(x, a_min=-0.2, a_max=0.4)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_leaky_relu(x_shape=(1, 8, 3, 3)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.nn.leaky_relu(x, alpha=0.1)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+    
+    def test_binary(op, x_shape, y_shape, y_is_const=False):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        if y_is_const:
+            y = relay.const(np.ones(y_shape).astype('float32'))
+            out = op(x, y)
+            f = relay.Function([x], out)
+            return f, {'x': x_shape}, []
+        y = relay.var('y', shape=(y_shape), dtype='float32')
+        out = op(x, y)
+        f = relay.Function([x, y], out)
+        return f, {'x': x_shape, 'y': y_shape}, []
+
+    def test_reduce(op, x_shape=(1, 2, 3, 4), axis=(2, 3), keepdims=False):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = op(x, axis=axis, keepdims=keepdims)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_strided_slice(x_shape, begin, end, strides=None):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        if strides:
+            out = relay.strided_slice(x, relay.expr.const(begin, dtype="int32"), relay.expr.const(end, dtype="int32"), relay.expr.const(strides, dtype="int32"))
+        else:
+            out = relay.strided_slice(x, relay.expr.const(begin, dtype="int32"), relay.expr.const(end, dtype="int32"))
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_adaptive_pool2d(op, x_shape=(1, 3, 32, 32), out_size=(1, 1)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = op(x, out_size)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_resize(x_shape=(1, 3, 16, 16), out_size=(32, 32), layout='NCHW', method='nearest_neighbor', coordinate_transformation_mode='align_corners'):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        out = relay.image.resize(x, out_size, layout=layout, method=method, coordinate_transformation_mode=coordinate_transformation_mode)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_multiple_outputs():
+        x = relay.var('x', shape=(1, 3), dtype='float32')
+        y = relay.var('y', shape=(1, 3), dtype='float32')
+        z = relay.add(x, y)
+        w = relay.add(z, y)
+        out = relay.Tuple((z, w))
+        f = relay.Function([x, y], out)
+        return f, {'x': (1, 3), 'y': (1, 3)}, []
+
+    def test_conv3d(x_shape=(1, 32, 8, 8, 8), k_shape=(16, 32, 3, 3, 3), groups=1, padding=(0, 0, 0), strides=(1, 1, 1), dilation=(1, 1, 1)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        out = relay.nn.conv3d(x, kernel, channels=k_shape[0], kernel_size=k_shape[2:], groups=groups, padding=padding, strides=strides, dilation=dilation)
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+
+    def test_pool3d(op, x_shape=(1, 3, 8, 32, 32), pool_size=(2, 2, 2), strides=(2, 2, 2), padding=(0, 0, 0), ceil_mode=False, count_include_pad=None):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        if count_include_pad is not None:
+            out = op(x, pool_size=pool_size, strides=strides, padding=padding, ceil_mode=ceil_mode, count_include_pad=count_include_pad)
+        else:
+            out = op(x, pool_size=pool_size, strides=strides, padding=padding, ceil_mode=ceil_mode)
+        f = relay.Function([x], out)
+        return f, {'x': x_shape}, []
+
+    def test_conv3d_transpose(x_shape=(1, 32, 8, 8, 8), k_shape=(32, 16, 3, 3, 3), groups=1, padding=(0, 0, 0), strides=(1, 1, 1), output_padding=(0, 0, 0)):
+        x = relay.var('x', shape=(x_shape), dtype='float32')
+        kernel = relay.var('kernel', shape=(k_shape), dtype='float32')
+        out = relay.nn.conv3d_transpose(x, kernel, channels=k_shape[1], kernel_size=k_shape[2:5], groups=groups, padding=padding, strides=strides, output_padding=output_padding)
+        f = relay.Function([x, kernel], out)
+        return f, {'x': x_shape, 'kernel': k_shape}, ['kernel']
+
+    run_and_verify(test_float_const())
+    run_and_verify(test_multiple_outputs())
+    run_and_verify(test_clip())
+    run_and_verify(test_leaky_relu())
+    run_and_verify(test_batch_norm((1, 64, 56, 56), (64,)))
+    run_and_verify(test_batch_norm((1, 56, 56, 64), (64,), axis=3, epsilon=1.001e-05))
+    run_and_verify(test_softmax((1, 1000), axis=1))
+    run_and_verify(test_softmax((1, 1000), axis=-1))
+    run_and_verify(test_softmax((1, 3, 4), axis=-2))
+    run_and_verify(test_softmax((1, 3, 4), axis=1))
+    for k_shape, groups in [((16, 32, 3, 3), 1), ((32, 1, 3, 3), 32)]:
+        for padding in [(0, 0), (1, 1)]:
+            for strides in [(1, 1), (2, 2)]:
+                for dilation in [(1, 1), (2, 2)]:
+                    run_and_verify(test_conv2d(k_shape=k_shape, groups=groups, padding=padding,
+                                               strides=strides, dilation=dilation))
+    run_and_verify(test_conv2d_nhwc())
+    run_and_verify(test_conv2d_const_weights())
+    run_and_verify(test_dense())
+    run_and_verify(test_dense_from_pytorch())
+    run_and_verify(test_bias_add())
+    run_and_verify(test_bias_add((1, 6, 3, 4), 6))
+    for op in [relay.add, relay.subtract, relay.multiply, relay.divide, relay.power]:
+        for y_is_const in [True, False]:
+            run_and_verify(test_binary(op, (1, 8, 3, 3), (1, 8, 3, 3), y_is_const))
+            run_and_verify(test_binary(op, (1, 8, 1, 3), (1, 8, 3, 1), y_is_const))
+            run_and_verify(test_binary(op, (1, 10), (10,), y_is_const))
+            run_and_verify(test_binary(op, (1, 1, 1, 10), (10,), y_is_const))
+            run_and_verify(test_binary(op, (1, 1, 1), (3,), y_is_const))
+    for pool_size in [(2, 2), (3, 3)]:
+        for strides in [(1, 1), (2, 2)]:
+            for padding in [(0, 0), (1, 1), (0, 0, 1, 1)]:
+                for ceil_mode in [False, True]:
+                    # Skip "the padding size is larger than or equal to the filter size for exclusive-counting pooling"
+                    if pool_size == (2, 2) and padding == (0, 0, 1, 1):
+                        continue
+                    for count_include_pad in [False, True]:
+                        # Skip "inclusive-counted blended or average pooling is not supported in combination with asymmetric padding"
+                        if count_include_pad and (padding == (0, 0, 1, 1) or strides == (2, 2)):
+                            continue
+                        run_and_verify(test_pool2d(relay.nn.avg_pool2d, pool_size=pool_size, strides=strides, padding=padding, ceil_mode=ceil_mode, count_include_pad=count_include_pad))
+                    run_and_verify(test_pool2d(relay.nn.max_pool2d, pool_size=pool_size, strides=strides, padding=padding, ceil_mode=ceil_mode))
+    for op in [relay.nn.global_max_pool2d, relay.nn.global_max_pool2d]:
+        run_and_verify(test_global_pool2d(op))
+    for op in [relay.nn.relu, relay.sigmoid, relay.tanh, relay.exp, relay.log, relay.sqrt,
+               relay.abs, relay.negative, relay.sin, relay.cos, relay.atan, relay.ceil, relay.floor]:
+        run_and_verify(test_unary(op))
+    run_and_verify(test_batch_flatten())
+    run_and_verify(test_expand_dims())
+    run_and_verify(test_squeeze((1, 5, 1, 1), (2, 3)))
+    run_and_verify(test_squeeze((1, 3, 1), (-1,)))
+    run_and_verify(test_concatenate([(1, 2, 6, 6), (1, 3, 6, 6)], axis=1))
+    for padding in [(0, 0), (1, 1)]:
+        for strides in [(1, 1), (2, 2)]:           
+                run_and_verify(test_conv2d_transpose(padding=padding, strides=strides))
+    run_and_verify(test_transpose((1, 16, 7, 7), [0, 2, 3, 1]))
+    run_and_verify(test_transpose((1, 7, 7, 16), [0, 3, 1, 2]))
+    run_and_verify(test_transpose_weights_conv2d())
+    run_and_verify(test_transpose_weights_conv2d((1, 32, 9, 9), (3, 3, 16, 32), (2, 3, 0, 1)))
+    run_and_verify(test_transpose_weights_dense())
+    run_and_verify(test_reshape((1, 1, 1, 10), (-1, 10)))
+    run_and_verify(test_reshape((1, 10, 2, 3), (1, -1)))
+    run_and_verify(test_reshape((1, 1, 2, 3), (1, 6)))
+    run_and_verify(test_pad((1, 8, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0]]))
+    run_and_verify(test_pad((1, 8, 16, 16), [[0, 0], [0, 0], [1, 1], [1, 1]]))
+    run_and_verify(test_pad((1, 8, 16, 16), [[0, 0], [0, 0], [0, 1], [2, 0]]))
+    run_and_verify(test_pad((1, 8, 3, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0], [0, 0]]))
+    for op in [relay.sum, relay.prod, relay.max, relay.min, relay.mean]:
+        for keepdims in [True, False]:
+            run_and_verify(test_reduce(op, axis=(1), keepdims=keepdims))
+            run_and_verify(test_reduce(op, axis=(2, 3), keepdims=keepdims))
+            run_and_verify(test_reduce(op, axis=(1, 2), keepdims=keepdims))
+            run_and_verify(test_reduce(op, axis=(1, 2, 3), keepdims=keepdims))
+    run_and_verify(test_strided_slice((1, 3, 6, 7), [0, 0, 0, 0], [1, 1, 6, 7]))
+    run_and_verify(test_strided_slice((1, 3, 6, 7), [0, 1, 0, 0], [1, 2, 6, 6]))
+    run_and_verify(test_strided_slice((1, 10), [0, 0], [1, 10], [1, 2]))
+    for op in [relay.nn.adaptive_max_pool2d, relay.nn.adaptive_avg_pool2d]:
+        run_and_verify(test_adaptive_pool2d(op))
+    run_and_verify(test_conv3d())
+    run_and_verify(test_conv3d(padding=(0, 0, 0, 1, 1, 1)))
+    run_and_verify(test_pool3d(relay.nn.avg_pool3d))
+    run_and_verify(test_pool3d(relay.nn.max_pool3d))
+    run_and_verify(test_pool3d(relay.nn.max_pool3d, padding=(0, 0, 0, 1, 1, 1)))
+    run_and_verify(test_pool3d(relay.nn.max_pool3d, strides=(1, 1, 1)))
+    run_and_verify(test_conv3d_transpose())
+    run_and_verify(test_conv3d_transpose(strides=(2, 2, 2)))
+    run_and_verify(test_conv3d_transpose(strides=(2, 2, 2), output_padding=(1, 1, 1)))
+
+def test_tensorrt_integration(test_all_models=False):
+    if should_skip():
+        return
+    
+    def test_model(model, i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+        def check_trt_used(graph):
+            import json
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum([1 for n in graph['nodes'] if n.get('attrs', {}).get('func_name', '').startswith('tensorrt_')])
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod = tensorrt.partition_for_tensorrt(mod, params)
+        with relay.build_config(opt_level=3):
+            graph, lib, params = relay.build(mod, "cuda", params=params)
+        if use_trt:
+            check_trt_used(graph)
+
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return latency, res
+
+    latency = {}
+    models = [
+        'alexnet',
+        'resnet18_v1',
+        'resnet18_v2',
+        'squeezenet1.0',
+        'mobilenet0.25',
+        'mobilenetv2_0.25',
+        'vgg11',
+        'densenet121',
+    ]
+    additional_models = [
+        'resnet34_v1',
+        'resnet50_v1',
+        'resnet101_v1',
+        'resnet152_v1',
+        'resnet34_v2',
+        'resnet50_v2',
+        'resnet101_v2',
+        'resnet152_v2',
+        'mobilenet0.5',
+        'mobilenet0.75',
+        'mobilenet1.0',
+        'mobilenetv2_0.5',
+        'mobilenetv2_0.75',
+        'mobilenetv2_1.0',
+        'vgg16',
+        'densenet169',
+        'densenet201']
+    if test_all_models:
+        models.extend(additional_models)
+    
+    dtype = 'float32'
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    for model in models:
+        latency[model], res = test_model(model, i_data, input_shape, dtype, use_trt=True)
+        _, ref_res = test_model(model, i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+        tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)

Review comment:
       Would be useful to know what model the test failed on

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/arm_compute_lib/acl_runtime.cc
+ * \brief A simple JSON runtime for Arm Compute Library.

Review comment:
       Should be TensorRT :)

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,573 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+def should_skip():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("skip because cuda is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("skip because tensorrt runtime is not available")
+        return True
+    return False
+
+def test_tensorrt_simple():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3,  1,  1)
+    zshape = (1,  1,  1,  1)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.var('y', shape=(yshape), dtype=dtype)
+    z = relay.var('z', shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_not_compatible():
+    if should_skip():
+        return
+    dtype = 'float32'
+    xshape = (1, 32, 14, 14)
+    x = relay.var('x', shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod['main'] = f
+    mod = tensorrt.partition_for_tensorrt(mod)
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod, "cuda")
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+def test_tensorrt_ops():
+    if should_skip():
+        return
+    def run_and_verify(config):
+        f, input_shapes, is_param = config
+        params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+        input_dict = {k: np.random.uniform(-1, 1, v).astype(np.float32) for k, v in input_shapes.items() if k not in is_param}
+
+        # Run TRT 
+        mod = tvm.IRModule()
+        mod['main'] = f
+        mod = tensorrt.partition_for_tensorrt(mod, params)
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**graph_params)
+        mod.run(**input_dict)
+        results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+        # Run reference
+        mod = tvm.IRModule()
+        mod['main'] = f
+        with relay.build_config(opt_level=3):
+            graph, lib, graph_params = relay.build(mod, "cuda", params=params)

Review comment:
       If a module cannot be built it would be useful to know which config it failed on

##########
File path: src/runtime/contrib/tensorrt/tensorrt_builder.cc
##########
@@ -0,0 +1,224 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_builder.cc
+ * \brief The TensorRTBuilder class can be used to convert a JSONRuntime graph into a TRT engine
+ * which can be used for inference.
+ */
+
+#include "tensorrt_builder.h"
+
+#include <tvm/runtime/ndarray.h>
+
+#include <memory>
+#include <string>
+
+#include "tensorrt_logger.h"
+#include "tensorrt_ops.h"
+#include "tensorrt_utils.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+TensorRTBuilder::TensorRTBuilder(TensorRTLogger* logger, size_t max_workspace_size,
+                                 bool use_implicit_batch, bool use_fp16, int batch_size)
+    : max_workspace_size_(max_workspace_size),
+      use_implicit_batch_(use_implicit_batch),
+      use_fp16_(use_fp16),
+      batch_size_(batch_size) {
+  // Create TRT builder and network.
+  builder_ = nvinfer1::createInferBuilder(*logger);
+#if TRT_VERSION_GE(6, 0, 1)
+  // Use INetworkV2.
+  auto flags =
+      1U << static_cast<uint32_t>(nvinfer1::NetworkDefinitionCreationFlag::kEXPLICIT_BATCH);
+  if (use_implicit_batch_) {
+    flags = 0U;
+    builder_->setMaxBatchSize(batch_size_);
+  }
+  network_ = builder_->createNetworkV2(flags);
+#else
+  // Use INetwork with implicit batch.
+  builder_->setMaxBatchSize(batch_size_);
+  builder_->setMaxWorkspaceSize(max_workspace_size_);
+  builder_->setFp16Mode(use_fp16_);
+  network_ = builder_->createNetwork();
+#endif
+}
+
+void TensorRTBuilder::AddInput(int nid, const JSONGraphNode& node) {
+  auto node_name = node.GetOpName();
+  auto shapes = node.GetOpShape();
+  auto dtypes = node.GetOpDataType();
+  CHECK_EQ(shapes.size(), dtypes.size());
+  node_output_map_[nid] = {};
+  for (size_t i = 0; i < shapes.size(); ++i) {
+    const std::string name = node_name + "_" + std::to_string(i);
+    auto shape = shapes[i];
+    // Remove batch dim when not in explicit batch mode.
+    if (use_implicit_batch_ && shape.size() > 1) {
+      shape.erase(shape.begin());
+    }
+    DLOG(INFO) << "TRT input: " << name << " " << DebugString(shape);
+    nvinfer1::Dims dims = VectorToTrtDims(shape);
+    CHECK(TypeMatch(dtypes[i], kDLFloat, 32)) << "Only FP32 inputs are supported.";
+    auto input_tensor = network_->addInput(name.c_str(), nvinfer1::DataType::kFLOAT, dims);
+    node_output_map_[nid].push_back(TrtOpInput(input_tensor));
+    network_input_names_.push_back(input_tensor->getName());
+  }
+}
+
+void TensorRTBuilder::AddConstant(int nid, const DLTensor* data) {
+  nvinfer1::Weights weight = GetDLTensorAsWeights(data, kDLCPU);
+  std::vector<int> shape(data->shape, data->shape + data->ndim);
+  // Remove batch dim when not in explicit batch mode.
+  if (use_implicit_batch_ && shape.size() > 1 && shape[0] == 1) {
+    shape.erase(shape.begin());
+  }
+  node_output_map_[nid] = {TrtOpInput(weight, shape)};
+}
+
+void TensorRTBuilder::AddOutput(const JSONGraphNodeEntry& node) {
+  auto it = node_output_map_.find(node.id_);
+  CHECK(it != node_output_map_.end()) << "Output was not found.";
+  auto out_tensor = it->second[node.index_].tensor;
+  std::string name = "tensorrt_output_" + std::to_string(network_output_names_.size());
+  out_tensor->setName(name.c_str());
+  network_->markOutput(*out_tensor);
+  network_output_names_.push_back(out_tensor->getName());
+  DLOG(INFO) << "TRT output: " << name << DebugString(TrtDimsToVector(out_tensor->getDimensions()));

Review comment:
       Same as above




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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-695094226


   cc @wpan11nv this might be something you or some of your Nvidia forks interested.


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

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



[GitHub] [incubator-tvm] trevor-m edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707258309


   Hey @areusch 
   I found that the failures for `Test MISRA-C runtime` are related to my new CMake option `USE_TENSORRT_CODEGEN`. When `USE_TENSORRT_CODEGEN` is `ON`, the CPP tests start to segfault. Since I turned it off, the CI passes. Can you see anything in my cmake rules that looks like it could be causing the issue?


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

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



[GitHub] [incubator-tvm] areusch commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
areusch commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-693512582


   looks like we are on cpplint 1.4.5. I tried running it at 1.5.4, latest release, but no change. There is an option in clang-format (AlignAfterOpenBracket) to control bracket alignment style, but the google style-guide setting it's at right now I think is the most reasonable. Further, I don't think cpplint can differentiate between #define continuations and regular code, so we can't configure that there.
   
   you could try manually fixing the line to make cpplint happy, then add on surrounding lines:
   `// clang-format off`
   `// clang-format on`
   
   this seems like an edge case in cpplint, so i'd vote for that.


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686739951


   Usually we need a separate PR to install the library/environment in the CI first (TRT in this case) so that we can have better e2e tests. 


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-712487066


   CI has passed with USE_TENSORRT_CODEGEN ON since the new CI container is used.


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

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



[GitHub] [incubator-tvm] comaniac edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
comaniac edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-686777770


   For the rest 2 points.
   
   2. Is that possible to move the pass before partitioning but after merge compiler region (like `PruneTesnorRTCompilerRegion`)? After the merge compiler region pass you should get the Relay graph with almost the same semantic as partitioning. If you could have a pass checking each compiler region for your constraints, you can probably just remove the region you don't want, so that you should get only valid partitioned functions.
   
   3. Can the TensorRT version be obtained via an API call in C++? Something like `tensorrt::get_version()`? If so you can register a global symbol and pass the version to Python so that it can be used by the annotator.
   
   ```python
   def conv2d(...):
       if not tvm.get_global_func("relay.tensorrt.version", True):
           return False
       ver = tvm.get_global_func("relay.tensorrt.version")
       if ver == '1.0':
           return True
       return False
   ```
   
   If you need manually set up the TensorRT version, then it could be like this: Let user specify it in `config.cmake` and we pass the value to a macro in C++ so that you could simply return the value. The drawback of this solution is that it needs to rebuild TVM to annotate different TensorRT versions, and I'm not sure if that makes sense to 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.

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692319247


   > > I have already created an API to retrieve the TRT version if TVM is compiled with the TRT runtime enabled. However, one of our use cases is to use TVM on a CPU-only instance to cross-compile models. For that use case, we want to be able to target compilation for different TRT versions - this affects the partitioning rules mostly. I don't think having to rebuild TVM for each target version will be a good solution.
   > > Is it possible for my annotation functions to access the pass context and therefore a TRT config that I will be adding as @masahi suggested? I don't see any other python code accessing the PassContext though...
   > 
   > Looks like `GetConfig` does not expose to the Python side.
   
   I see, in that case I think my current implementation using a global variable is fine since it is all confined within the one file.
   
   
   > Zhi just pointed to me offline about how to access pass context configs in Python. Here is an example:
   > 
   > ```python
   > import tvm
   > with tvm.transform.PassContext(config={"relay.fallback_device_type": 5}):
   >     pass_ctx = tvm.transform.PassContext.current()
   >     print(pass_ctx.config["relay.fallback_device_type"])
   > ```
   
   Nice! Let me try that.


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

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



[GitHub] [incubator-tvm] zhiics merged pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
zhiics merged pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395


   


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-692256923


   Thanks everyone for the great feedback! I've been busy lately, but I plan to start addressing the comments this week.


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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



##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       The message is a bit confusing. `USE_TENSORRT` means enabling the TensorRT codegen for graph partitininog. It doesn't require TensorRT to be available in the system environment. IIUC, maybe it's better to say "Build with TensorRT codegen", although I just found that "Build with Arm Compute Library" has the same issue.
   
   @lhutton1 could you also share your thoughts for this?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch

Review comment:
       Do we need a guard here as in L52?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,671 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+import os
+import logging
+import numpy as np
+
+# Version to use for annotation when there is no linked TRT.
+#TENSORRT_VERSION = (6, 0, 1)
+#USE_IMPLICIT_BATCH = True
+#REMOVE_NO_MAC_SUBGRAPHS = False
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+def partition_for_tensorrt(mod, params=None, version=None, use_implicit_batch=True,
+                           remove_no_mac_subgraphs=False, max_workspace_size=1 << 30):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warn("TVM was not built against TensorRT and no version was provided to partition_for_tensorrt. Defaulting to 6.0.1")
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+    seq = tvm.transform.Sequential([transform.InferType(),
+                                    RemoveDropoutPass(),
+                                    transform.RemoveUnusedFunctions(),
+                                    transform.ConvertLayout({'nn.conv2d': ['NCHW', 'default'],
+                                                             'nn.conv3d': ['NCDHW', 'default']}),
+                                    transform.FoldConstant(),
+                                    transform.AnnotateTarget('tensorrt'),
+                                    transform.MergeCompilerRegions(),
+                                    transform.PartitionGraph(),
+                                    transform.InferType()])
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            print("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode() and \
+            (isinstance(args[0], Constant) or isinstance(args[1], Constant)) and \
+            args[0].checked_type.shape[0] == args[1].checked_type.shape[0] and \
+            args[0].checked_type.shape[0] != 1 and \
+            (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3):
+        print("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        print("nn.batch_norm: axis is {} but must be 1 or 3.".format(int(attrs.axis)))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d: data_layout is {} but must be NCHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d: kernel_layout is {} but must be OIHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d: out_layout is {} but must be NCHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.dense: input has rank {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    if weight_rank != 2:
+        print("nn.dense: weight has rank {} but must be 2.".format(weight_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        print("nn.bias_add: input rank is {} but must be 2, 3 or 4.".format(input_rank))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        print("nn.avg_pool2d: inclusive-counted blended or average "
+                "pooling is not supported in combination with asymmetric padding")
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        print("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_max_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        print("nn.global_avg_pool2d: layout is {} but must be NCHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        print("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        print("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        print("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        print("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                print("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        print("nn.conv2d_transpose: data_layout is {} but must be NCHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIHW":
+        print("nn.conv2d_transpose: kernel_layout is {} but must be OIHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        print("nn.conv2d_transpose: out_layout is {} but must be NCHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        print("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def resize_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [("NCHW", "NHWC"), ("NHWC", "NCHW"), ("NDHWC", "NCDHW"), ("NCDHW", "NDHWC")]:
+        print("layout_transform: {} to {} is not supported.".format(attrs.src_layout, attrs.dst_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        print("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            print("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            print("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        print("nn.pad: pad mode is {} but must be constant.".format(attrs.pad_mode))
+        return False
+    if float(attrs.pad_value) != 0.0:
+        print("nn.pad: pad value is {} but must be 0.0.".format(float(attrs.pad_value)))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        print("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        print("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+def reduce_annotate_fn(attrs, args, op_name):
+    if not attrs.axis or len(attrs.axis) == 0:
+        print("{}: cannot reduce to scalar.".format(op_name))
+        return False
+    if attrs.exclude:
+        print("{}: exclude not supported.".format(op_name))
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        print("{}: can't modify batch dimension.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    if get_tensorrt_version() < (5, 1, 5):
+        print("{}: requires TensorRT version 5.1.5 or higher.".format(op_name))
+        return False
+    return True
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if args[0].checked_type.dtype != "float32":
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = attrs.end[0] is not None and int(attrs.end[0]) != -1 and \
+                                    int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            print("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        print("strided_slice: stride must be positive")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        print("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.upsampling", "target.tensorrt")
+def upsampling_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    # TODO(trevmorr): Output does not match TVM. Disable.
+    return False
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d: data_layout is {} but must be NCDHW.".format(attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d: kernel_layout is {} but must be OIDHW.".format(attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d: out_layout is {} but must be NCDHW.".format(attrs.out_layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.max_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        print("nn.avg_pool3d: layout is {} but must be NCDHW.".format(attrs.layout))
+        return False
+    return True
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args): # pylint: disable=unused-variable
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        print("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        print("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        print("nn.conv3d_transpose: data_layout is {} but must be NCDHW.".format(
+            attrs.data_layout))
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        print("nn.conv3d_transpose: kernel_layout is {} but must be OIDHW.".format(
+            attrs.kernel_layout))
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        print("nn.conv3d_transpose: out_layout is {} but must be NCDHW.".format(
+            attrs.out_layout))
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        print("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        print("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+def is_invalid_subgraph(params, body):
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        print('tensorrt: scalar inputs not supported')
+                        return True
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    print('tensorrt: scalar inputs not supported')
+                    return True
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and \
+           any([x != input_batch_sizes[0] for x in input_batch_sizes[1:]]):
+            print('tensorrt: inputs have different batch sizes')
+            return True
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return True
+    return False
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):
+    class VarReplacer(ExprMutator):
+        """
+        Visit an expression while replacing vars according to var_map. Used by
+        SubgraphRemover/PruneSubgraphs to return a subgraph originally partitioned to TRT back to TVM.
+        """
+        def __init__(self, var_map):
+            ExprMutator.__init__(self)
+            self.var_map = var_map
+
+        def visit_var(self, var):
+            if var in self.var_map:
+                return self.var_map[var]
+            return super().visit_var(var)
+
+    class SubgraphRemover(ExprMutator):

Review comment:
       This is the issue we discussed in this PR about how to deal with post-partitioning judgements. We could later on figure out an approach to generalize this requirement.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]

Review comment:
       ```suggestion
       ret: Tuple[int, int, int]
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]

Review comment:
       ```suggestion
       version : Optional[Tuple[int, int, int]]
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.

Review comment:
       Shoud be fine in one line.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs

Review comment:
       Do we need a guard here as in L52?

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper

Review comment:
       These two functions share the same logic so it would be better to merge them:
   
   ```python
   def _register_external_op_helper_with_checker(op_name, checker):
       # The body of _register_external_op_helper_func
   
   def _register_external_op_helper(op_name, supported=True):
       return _register_external_op_helper_with_checker(op_name, lambda attrs, args, op_name: supported)
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.

Review comment:
       ```suggestion
       mod_n_config: Tuple[Module, Dict[str, Any]]
           A tuple of 1) annotated and partitioned module and 2) "relay.ext.tensorrt.options" configuration
           which should be given to PassContext when building.
   ```

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")

Review comment:
       - Use the standard logger: `logger = logging.getLogger("TensorRT")`.
   - "info" seems not a right log level for messages like this. Should be "warn" or "debug".
   - Ditto to other logs.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.

Review comment:
       s/relay/Relay/g

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================

Review comment:
       ```suggestion
   ==========================
   ```

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.

Review comment:
       Better to provide an overview of this tutorial, including what information/takeaway readers could expect after reading this article.
   
   After reading through, I would suggest mentioning the following points:
   1. (optional) Some model performance numbers to impressive readers.
   2. Install TensorRT and build TVM with it.
   3. Build a ResNet-18 with TensorRT.
   4. Operator support (I also suggest moving this section to the end along with the last section about adding new operators.)
   5. Deploy the built model with TensorRT runtime.
   6. How to support other operators.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------

Review comment:
       ```suggestion
   -------------------
   ```
   Note that misaligning will cause doc generation warnings and failed the CI.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.

Review comment:
       Better to say this section is tested with TensorRT version XXX and may not stay up-to-date with newer versions that changes the APIs.

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (
+        not get_tensorrt_use_implicit_batch_mode()
+        and (isinstance(args[0], Constant) or isinstance(args[1], Constant))
+        and args[0].checked_type.shape[0] == args[1].checked_type.shape[0]
+        and args[0].checked_type.shape[0] != 1
+        and (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3)
+    ):
+        logging.info("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.batch_norm is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        logging.info("nn.batch_norm: axis is %d but must be 1 or 3.", int(attrs.axis))
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.softmax is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info("nn.conv2d: kernel_layout is %s but must be OIHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if dense is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.dense: input has rank %d but must be 2, 3 or 4.", input_rank)
+        return False
+    if weight_rank != 2:
+        logging.info("nn.dense: weight has rank %d but must be 2.", weight_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.bias_add is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.bias_add: input rank is %d but must be 2, 3 or 4.", input_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.avg_pool2d: layout is %d but must be NCHW.", attrs.layout)
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        logging.info(
+            "nn.avg_pool2d: inclusive-counted blended or average "
+            "pooling is not supported in combination with asymmetric padding"
+        )
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_avg_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if expand_dims is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if squeeze is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        logging.info("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        logging.info("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if concatenate is supported by TensorRT."""
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        logging.info("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                logging.info("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info(
+            "nn.conv2d_transpose: kernel_layout is %s but must be OIHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        logging.info("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def layout_transform_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if layout_transform is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [
+        ("NCHW", "NHWC"),
+        ("NHWC", "NCHW"),
+        ("NDHWC", "NCDHW"),
+        ("NCDHW", "NDHWC"),
+    ]:
+        logging.info(
+            "layout_transform: %s to %s is not supported.", attrs.src_layout, attrs.dst_layout
+        )
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if reshape is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        logging.info("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            logging.info("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            logging.info("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.pad is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        logging.info("nn.pad: pad mode is %s but must be constant.", attrs.pad_mode)
+        return False
+    if float(attrs.pad_value) != 0.0:
+        logging.info("nn.pad: pad value is %f but must be 0.0.", float(attrs.pad_value))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        logging.info("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        logging.info("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+
+def reduce_annotate_fn(attrs, args, op_name):
+    """Helper for reduce operations."""
+    if not attrs.axis or len(attrs.axis) == 0:
+        logging.info("%s: cannot reduce to scalar.", op_name)
+        return False
+    if attrs.exclude:
+        logging.info("%s: exclude not supported.", op_name)
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        logging.info("%s: can't modify batch dimension.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    """Helper for ops which require TRT 5.1.5 or greater."""
+    if get_tensorrt_version() < (5, 1, 5):
+        logging.info("%s: requires TensorRT version 5.1.5 or higher.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if strided_slice is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = (
+            attrs.end[0] is not None
+            and int(attrs.end[0]) != -1
+            and int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        )
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            logging.info("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        logging.info("strided_slice: stride must be positive")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):

Review comment:
       It might be better to generalize `trt_5_1_5_annotate_fn` to `trt_version_annotate_fn(version, attrs, args, op_name)`.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph from a MXNet ResNet18 model.
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    block = get_model('resnet18_v1', pretrained=True)
+    mod, params = relay.frontend.from_mxnet(block, shape={'data': input_shape}, dtype=dtype)
+
+
+Annotate and partition the graph for TensorRT. All ops which are supported by the TensorRT
+integration will be marked and offloaded to TensorRT. The rest of the ops will go through the
+regular TVM CUDA compilation and code generation.
+
+.. code:: python
+
+    from tvm.relay.op.contrib.tensorrt import partition_for_tensorrt
+    mod, config = partition_for_tensorrt(mod, params)
+
+
+Build the Relay graph, using the new module and config returned by partition_for_tensorrt. The
+target must always be "cuda".

Review comment:
       This statement is not 100% correct. It can also be "cuda -model=v100" for example.

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------
+
+In order to download TensorRT, you will need to create an NVIDIA Developer program account. Please
+see NVIDIA's documentation for more info:
+https://docs.nvidia.com/deeplearning/tensorrt/install-guide/index.html. If you have a Jetson device
+such as a TX1, TX2, Xavier, or Nano, TensorRT will already be installed on the device via the
+JetPack SDK.
+
+There are two methods to install TensorRT:
+
+* System install via deb or rpm package.
+* Tar file installation.
+
+With the tar file installation method, you must provide the path of the extracted tar archive to
+USE_TENSORT_GRAPH_RUNTIME=/path/to/TensorRT. With the system install method,
+USE_TENSORT_GRAPH_RUNTIME=ON will automatically locate your installation.
+
+Building TVM with TensorRT support
+----------------------------------
+
+There are two separate build flags for TensorRT integration in TVM:
+
+* USE_TENSORT=ON/OFF - This flag will enable compiling a TensorRT module, which does not require any
+TensorRT library.
+* USE_TENSORT_GRAPH_RUNTIME=ON/OFF/path-to-TensorRT - This flag will enable the TensorRT runtime
+module. This will build TVM against the TensorRT libraries.
+
+Example setting in config.cmake file:
+
+.. code:: cmake
+
+    set(USE_TENSORRT ON)
+    set(USE_TENSORRT_GRAPH_RUNTIME /home/ubuntu/TensorRT-7.0.0.11)
+
+
+Usage

Review comment:
       Explicitly say what you are doing in this section, such as "Build and Deploy ResNet-18 with TensorRT"

##########
File path: python/tvm/relay/op/contrib/tensorrt.py
##########
@@ -0,0 +1,751 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""TensorRT supported operators."""
+import logging
+import numpy as np
+import tvm
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.expr import Call, Constant, Tuple, GlobalVar
+from tvm.relay.expr_functor import ExprMutator
+
+
+def is_tensorrt_runtime_enabled():
+    """Check if the TensorRT graph runtime is present.
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_tensorrt_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def get_tensorrt_version():
+    """Gets the version of TensorRT that TVM is built against or is targeting.
+
+    Returns
+    -------
+    ret: Tuple[int]
+        TensorRT version as a tuple of major, minor, and patch number. If TVM
+        is not built with TensorRT, the value set by set_tensorrt_version() is returned instead.
+    """
+    pass_ctx = tvm.transform.PassContext.current()
+    if "relay.ext.tensorrt.options" in pass_ctx.config:
+        return tuple(pass_ctx.config["relay.ext.tensorrt.options"].tensorrt_version)
+    return tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+
+
+def get_tensorrt_use_implicit_batch_mode():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].use_implicit_batch
+
+
+def get_tensorrt_remove_no_mac_subgraphs():
+    pass_ctx = tvm.transform.PassContext.current()
+    return pass_ctx.config["relay.ext.tensorrt.options"].remove_no_mac_subgraphs
+
+
+def partition_for_tensorrt(
+    mod,
+    params=None,
+    version=None,
+    use_implicit_batch=True,
+    remove_no_mac_subgraphs=False,
+    max_workspace_size=1 << 30,
+):
+    """Partition the graph greedily offloading supported
+    operators to TensorRT.
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+    version : Optional[Tuple(int)]
+        TensorRT version to target as tuple of (major, minor, patch). If TVM is compiled with
+        USE_TENSORRT_GRAPH_RUNTIME=ON, the linked TensorRT version will be used instead.
+    use_implicit_batch : Optional[bool]
+        Use TensorRT implicit batch mode (default true). Setting to false will enable explicit batch
+        mode which will widen supported operators to include those which modify the batch dimension,
+        but may reduce performance for some models.
+    remove_no_mac_subgraphs : Optional[bool]
+        Removes subgraphs which have been partitioned for TensorRT if they do not have any
+        multiply-accumulate operations. The removed subgraphs will go through TVM's standard
+        compilation instead. Can improve performance.
+    max_workspace_size : Optional[int]
+        How many bytes of workspace size to allow each subgraph to use for TensorRT engine creation.
+        See TensorRT documentation for more info.
+    Returns
+    -------
+    mod : annotated and partitioned module.
+    config : "relay.ext.tensorrt.options" configuration which should be given to PassContext when
+             building.
+    """
+    config = {
+        "use_implicit_batch": use_implicit_batch,
+        "max_workspace_size": max_workspace_size,
+        "remove_no_mac_subgraphs": remove_no_mac_subgraphs,
+    }
+    if version:
+        assert isinstance(version, tuple) and len(version) == 3
+        config["tensorrt_version"] = version
+    else:
+        linked_version = tuple(tvm.get_global_func("relay.op.get_tensorrt_version")())
+        if not linked_version:
+            logging.warning(
+                "TVM was not built against TensorRT and no version was provided to "
+                "partition_for_tensorrt. Defaulting to 6.0.1"
+            )
+            linked_version = (6, 0, 1)
+        config["tensorrt_version"] = linked_version
+
+    if params:
+        mod["main"] = bind_params_by_name(mod["main"], params)
+    seq = tvm.transform.Sequential(
+        [
+            transform.InferType(),
+            RemoveDropoutPass(),
+            transform.RemoveUnusedFunctions(),
+            transform.ConvertLayout(
+                {"nn.conv2d": ["NCHW", "default"], "nn.conv3d": ["NCDHW", "default"]}
+            ),
+            transform.FoldConstant(),
+            transform.AnnotateTarget("tensorrt"),
+            transform.MergeCompilerRegions(),
+            transform.PartitionGraph(),
+            transform.InferType(),
+        ]
+    )
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        mod = seq(mod)
+        mod = prune_tensorrt_subgraphs(mod)
+    return mod, config
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return supported
+
+    return _func_wrapper
+
+
+def _register_external_op_helper_func(op_name, func):
+    @tvm.ir.register_op_attr(op_name, "target.tensorrt")
+    def _func_wrapper(attrs, args):
+        if any([x.checked_type.dtype != "float32" for x in args]):
+            logging.info("Only float32 inputs are supported for TensorRT.")
+            return False
+        return func(attrs, args, op_name)
+
+    return _func_wrapper
+
+
+# Ops which are always supported
+_register_external_op_helper("nn.relu")
+_register_external_op_helper("sigmoid")
+_register_external_op_helper("tanh")
+_register_external_op_helper("subtract")
+_register_external_op_helper("multiply")
+_register_external_op_helper("divide")
+_register_external_op_helper("power")
+_register_external_op_helper("maximum")
+_register_external_op_helper("minimum")
+_register_external_op_helper("exp")
+_register_external_op_helper("log")
+_register_external_op_helper("sqrt")
+_register_external_op_helper("abs")
+_register_external_op_helper("negative")
+_register_external_op_helper("nn.batch_flatten")
+_register_external_op_helper("clip")
+
+
+@tvm.ir.register_op_attr("add", "target.tensorrt")
+def add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if add is supported by TensorRT."""
+
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (
+        not get_tensorrt_use_implicit_batch_mode()
+        and (isinstance(args[0], Constant) or isinstance(args[1], Constant))
+        and args[0].checked_type.shape[0] == args[1].checked_type.shape[0]
+        and args[0].checked_type.shape[0] != 1
+        and (len(args[0].checked_type.shape) > 3 or len(args[1].checked_type.shape) > 3)
+    ):
+        logging.info("add: bug in TRT with adding batched constants.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.batch_norm", "target.tensorrt")
+def batch_norm_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.batch_norm is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if int(attrs.axis) not in (1, 3):
+        logging.info("nn.batch_norm: axis is %d but must be 1 or 3.", int(attrs.axis))
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.softmax", "target.tensorrt")
+def softmax_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.softmax is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("nn.softmax: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.tensorrt")
+def conv2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info("nn.conv2d: kernel_layout is %s but must be OIHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.dense", "target.tensorrt")
+def dense_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if dense is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    weight_rank = len(args[1].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.dense: input has rank %d but must be 2, 3 or 4.", input_rank)
+        return False
+    if weight_rank != 2:
+        logging.info("nn.dense: weight has rank %d but must be 2.", weight_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.bias_add", "target.tensorrt")
+def bias_add_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.bias_add is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    input_rank = len(args[0].checked_type.shape)
+    if input_rank not in (2, 3, 4):
+        logging.info("nn.bias_add: input rank is %d but must be 2, 3 or 4.", input_rank)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool2d", "target.tensorrt")
+def max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool2d", "target.tensorrt")
+def avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.avg_pool2d: layout is %d but must be NCHW.", attrs.layout)
+        return False
+    if attrs.count_include_pad and len(attrs.padding) == 4:
+        logging.info(
+            "nn.avg_pool2d: inclusive-counted blended or average "
+            "pooling is not supported in combination with asymmetric padding"
+        )
+        return False
+    if attrs.ceil_mode and get_tensorrt_version() < (5, 1, 5):
+        logging.info("nn.avg_pool2d: ceil_mode=True requires TensorRT 5.1.5 or greater.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_max_pool2d", "target.tensorrt")
+def global_max_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_max_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.global_avg_pool2d", "target.tensorrt")
+def global_avg_pool_2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.global_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.layout != "NCHW":
+        logging.info("nn.global_avg_pool2d: layout is %s but must be NCHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("expand_dims", "target.tensorrt")
+def expand_dims_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if expand_dims is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axis) == 0:
+        logging.info("expand_dims: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("squeeze", "target.tensorrt")
+def squeeze_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if squeeze is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not attrs.axis:
+        logging.info("squeeze: must explicitly set axis.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([axis == 0 for axis in map(int, attrs.axis)]):
+        logging.info("squeeze: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("concatenate", "target.tensorrt")
+def concatenate_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if concatenate is supported by TensorRT."""
+    if any([x.dtype != "float32" for x in args[0].checked_type.fields]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not get_tensorrt_use_implicit_batch_mode():
+        return True
+    if int(attrs.axis) == 0:
+        logging.info("concatenate: can't modify batch dimension.")
+        return False
+    if isinstance(args[0], Tuple):
+        for tuple_input in args[0].fields:
+            if isinstance(tuple_input, Constant):
+                logging.info("concatenate: can't concatenate tensors with constants.")
+                return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv2d_transpose", "target.tensorrt")
+def conv2d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv2d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.data_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: data_layout is %s but must be NCHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIHW":
+        logging.info(
+            "nn.conv2d_transpose: kernel_layout is %s but must be OIHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCHW":
+        logging.info("nn.conv2d_transpose: out_layout is %s but must be NCHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv2d_transpose: dilation rate must be 1.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("transpose", "target.tensorrt")
+def transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and int(attrs.axes[0]) != 0:
+        logging.info("transpose: can't modify batch dimension.")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("layout_transform", "target.tensorrt")
+def layout_transform_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if layout_transform is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if (attrs.src_layout, attrs.dst_layout) not in [
+        ("NCHW", "NHWC"),
+        ("NHWC", "NCHW"),
+        ("NDHWC", "NCDHW"),
+        ("NCDHW", "NDHWC"),
+    ]:
+        logging.info(
+            "layout_transform: %s to %s is not supported.", attrs.src_layout, attrs.dst_layout
+        )
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("reshape", "target.tensorrt")
+def reshape_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if reshape is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if any([x < -1 for x in map(int, attrs.newshape)]):
+        logging.info("reshape: new shape dims must be explicit.")
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        shape = list(map(int, args[0].checked_type.shape))
+        new_shape = list(map(int, attrs.newshape))
+        if len(new_shape) == 0 or len(shape) == 0:
+            logging.info("reshape: Can't reshape to or from scalar.")
+            return False
+        # TRT cannot modify batch dimension.
+        original_volume = np.prod(shape)
+        # First, resolve 0.
+        for i, value in enumerate(new_shape):
+            if value == 0:
+                new_shape[i] = shape[i]
+        # Resolve -1.
+        for i, value in enumerate(new_shape):
+            if value == -1:
+                new_shape[i] = original_volume // np.prod([x for x in new_shape if x != -1])
+        if shape[0] != new_shape[0]:
+            logging.info("reshape: can't modify batch dimension.")
+            return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.pad", "target.tensorrt")
+def pad_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.pad is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if attrs.pad_mode != "constant":
+        logging.info("nn.pad: pad mode is %s but must be constant.", attrs.pad_mode)
+        return False
+    if float(attrs.pad_value) != 0.0:
+        logging.info("nn.pad: pad value is %f but must be 0.0.", float(attrs.pad_value))
+        return False
+    if any([x != 0 for x in attrs.pad_width[0]]) or any([x != 0 for x in attrs.pad_width[1]]):
+        logging.info("nn.pad: can't pad batch or channel dimensions.")
+        return False
+    if len(attrs.pad_width) == 5 and any([x != 0 for x in attrs.pad_width[2]]):
+        logging.info("nn.pad: can only pad last two dimensions for 5D inputs.")
+    return True
+
+
+def reduce_annotate_fn(attrs, args, op_name):
+    """Helper for reduce operations."""
+    if not attrs.axis or len(attrs.axis) == 0:
+        logging.info("%s: cannot reduce to scalar.", op_name)
+        return False
+    if attrs.exclude:
+        logging.info("%s: exclude not supported.", op_name)
+        return False
+    if get_tensorrt_use_implicit_batch_mode() and any([x == 0 for x in map(int, attrs.axis)]):
+        logging.info("%s: can't modify batch dimension.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("sum", reduce_annotate_fn)
+_register_external_op_helper_func("prod", reduce_annotate_fn)
+_register_external_op_helper_func("max", reduce_annotate_fn)
+_register_external_op_helper_func("min", reduce_annotate_fn)
+_register_external_op_helper_func("mean", reduce_annotate_fn)
+
+
+def trt_5_1_5_annotate_fn(attrs, args, op_name):
+    """Helper for ops which require TRT 5.1.5 or greater."""
+    if get_tensorrt_version() < (5, 1, 5):
+        logging.info("%s: requires TensorRT version 5.1.5 or higher.", op_name)
+        return False
+    return True
+
+
+_register_external_op_helper_func("nn.leaky_relu", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("sin", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("cos", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("atan", trt_5_1_5_annotate_fn)
+_register_external_op_helper_func("ceil", trt_5_1_5_annotate_fn)
+
+
+@tvm.ir.register_op_attr("strided_slice", "target.tensorrt")
+def strided_slice_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if strided_slice is supported by TensorRT."""
+    if args[0].checked_type.dtype != "float32":
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if not trt_5_1_5_annotate_fn(attrs, args, "strided_slice"):
+        return False
+    if get_tensorrt_use_implicit_batch_mode():
+        batch_dim_begin_modified = attrs.begin[0] is not None and int(attrs.begin[0]) != 0
+        batch_dim_end_modified = (
+            attrs.end[0] is not None
+            and int(attrs.end[0]) != -1
+            and int(attrs.end[0]) != int(args[0].checked_type.shape[0])
+        )
+        if batch_dim_begin_modified or batch_dim_end_modified:
+            logging.info("strided_slice: can't modify batch dimension.")
+            return False
+    if any([x is not None and x <= 0 for x in attrs.strides]):
+        logging.info("strided_slice: stride must be positive")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_max_pool2d", "target.tensorrt")
+def adapative_max_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_max_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_max_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.adaptive_avg_pool2d", "target.tensorrt")
+def adapative_avg_pool2d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.adaptive_avg_pool2d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if len(attrs.output_size) == 0 or any([size != 1 for size in map(int, attrs.output_size)]):
+        logging.info("nn.adaptive_avg_pool2d: output size must be (1, 1).")
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d", "target.tensorrt")
+def conv3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.conv3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        logging.info("nn.conv3d: data_layout is %s but must be NCDHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        logging.info("nn.conv3d: kernel_layout is %s but must be OIDHW.", attrs.kernel_layout)
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        logging.info("nn.conv3d: out_layout is %s but must be NCDHW.", attrs.out_layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.max_pool3d", "target.tensorrt")
+def max_pool_3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.max_pool3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.max_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        logging.info("nn.max_pool3d: layout is %s but must be NCDHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.avg_pool3d", "target.tensorrt")
+def avg_pool_3d_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.avg_pool3d is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.avg_pool3d: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.layout != "NCDHW":
+        logging.info("nn.avg_pool3d: layout is %s but must be NCDHW.", attrs.layout)
+        return False
+    return True
+
+
+@tvm.ir.register_op_attr("nn.conv3d_transpose", "target.tensorrt")
+def conv3d_transpose_annotate_fn(attrs, args):  # pylint: disable=unused-variable
+    """Check if nn.conv3d_transpose is supported by TensorRT."""
+    if any([x.checked_type.dtype != "float32" for x in args]):
+        logging.info("Only float32 inputs are supported for TensorRT.")
+        return False
+    if get_tensorrt_version() < (6, 0, 1):
+        logging.info("nn.conv3d_transpose: requires TensorRT version 6.0.1 or higher.")
+        return False
+    if attrs.data_layout != "NCDHW":
+        logging.info("nn.conv3d_transpose: data_layout is %s but must be NCDHW.", attrs.data_layout)
+        return False
+    if attrs.kernel_layout != "OIDHW":
+        logging.info(
+            "nn.conv3d_transpose: kernel_layout is %s but must be OIDHW.", attrs.kernel_layout
+        )
+        return False
+    if attrs.out_layout and attrs.out_layout != "NCDHW":
+        logging.info("nn.conv3d_transpose: out_layout is %s but must be NCDHW.", attrs.out_layout)
+        return False
+    if attrs.dilation and any([rate != 1 for rate in map(int, attrs.dilation)]):
+        logging.info("nn.conv3d_transpose: dilation rate must be 1.")
+        return False
+    if attrs.output_padding and any([x != 0 for x in map(int, attrs.output_padding)]):
+        logging.info("nn.conv3d_transpose: output padding is not supported.")
+        return False
+    return True
+
+
+def is_valid_subgraph(params, body):
+    """Final check on whether the subgraph is valid and should be offloaded to TensorRT."""
+    # Remove invalid subgraphs for implicit batch mode.
+    if get_tensorrt_use_implicit_batch_mode():
+        input_batch_sizes = []
+        for var in params:
+            # In implicit batch mode, all inputs must have same batch size
+            if isinstance(var.checked_type, relay.TupleType):
+                for tupe_type in var.checked_type.fields:
+                    # Scalar inputs not allowed
+                    if len(tupe_type.shape) == 0:
+                        logging.info("tensorrt: scalar inputs not supported")
+                        return False
+                    input_batch_sizes.append(int(tupe_type.shape[0]))
+            else:
+                # Scalar inputs not allowed
+                if len(var.checked_type.shape) == 0:
+                    logging.info("tensorrt: scalar inputs not supported")
+                    return False
+                input_batch_sizes.append(int(var.checked_type.shape[0]))
+        if len(input_batch_sizes) > 1 and len(set(input_batch_sizes)) != 1:
+            logging.info("tensorrt: inputs have different batch sizes")
+            return False
+    # Remove subgraphs with no multiply-accumulates
+    if get_tensorrt_remove_no_mac_subgraphs() and relay.analysis.get_total_mac_number(body) == 0:
+        return False
+    return True
+
+
+def prune_tensorrt_subgraphs(mod, target="tensorrt"):

Review comment:
       As it is already prune_"tensorrt"_subgraphs, it looks to me that you don't need to specify `target` in the arguments?

##########
File path: CMakeLists.txt
##########
@@ -76,6 +76,8 @@ tvm_option(USE_COREML "Build with coreml support" OFF)
 tvm_option(USE_TARGET_ONNX "Build with ONNX Codegen support" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB "Build with Arm Compute Library" OFF)
 tvm_option(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME "Build with Arm Compute Library graph runtime" OFF)
+tvm_option(USE_TENSORRT "Build with TensorRT" OFF)

Review comment:
       The message is a bit confusing. `USE_TENSORRT` means enabling the TensorRT codegen for graph partitininog. It doesn't require TensorRT to be available in the system environment. IIUC, maybe it's better to say "Build with TensorRT codegen", although I just found that "Build with Arm Compute Library" has the same issue.
   
   @lhutton1 could you also share your thoughts about this?

##########
File path: docs/deploy/tensorrt.rst
##########
@@ -0,0 +1,267 @@
+..  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.
+
+Relay TensorRT Integration
+==============================================
+**Author**: `Trevor Morris <https://github.com/trevor-m>`_
+
+Introduction
+------------
+
+NVIDIA TensorRT is a library for optimized deep learning inference. This integration will offload as
+many operators as possible from Relay to TensorRT, providing a performance boost on NVIDIA GPUs
+without the need to tune schedules.
+
+Installing TensorRT
+------------------------------

Review comment:
       ```suggestion
   -------------------
   ```
   Note that misaligning will cause doc generation warnings and fail the CI.

##########
File path: src/relay/backend/contrib/tensorrt/codegen.cc
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/tensorrt/codegen.cc
+ * \brief Implementation of the TensorRT JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../utils.h"
+#include "../codegen_json/codegen_json.h"
+
+#if TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#endif
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*! \brief Attributes to store the compiler options for TensorRT. */
+struct TensorRTCompilerConfigNode : public tvm::AttrsNode<TensorRTCompilerConfigNode> {
+  Array<Integer> tensorrt_version;
+  bool use_implicit_batch;
+  size_t max_workspace_size;
+  bool remove_no_mac_subgraphs;
+
+  TVM_DECLARE_ATTRS(TensorRTCompilerConfigNode, "ext.attrs.TensorRTCompilerConfigNode") {
+    TVM_ATTR_FIELD(tensorrt_version)
+        .describe("TensorRT version as (major, minor, patch).")
+        .set_default(Array<Integer>({6, 0, 1}));
+    TVM_ATTR_FIELD(use_implicit_batch).set_default(true);
+    TVM_ATTR_FIELD(max_workspace_size).set_default(size_t(1) << 30);
+    TVM_ATTR_FIELD(remove_no_mac_subgraphs).set_default(false);
+  }
+};
+
+class TensorRTCompilerConfig : public Attrs {
+ public:
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TensorRTCompilerConfig, Attrs,
+                                            TensorRTCompilerConfigNode);
+};
+
+TVM_REGISTER_NODE_TYPE(TensorRTCompilerConfigNode);
+TVM_REGISTER_PASS_CONFIG_OPTION("relay.ext.tensorrt.options", TensorRTCompilerConfig);
+
+/*!
+ * \brief Generates an TensorRTModule from a relay expression by serializing the expression to a
+ * json representation. TensorRT is not required here because use of TensorRT APIs is deferred until
+ * runtime.
+ */
+class TensorRTJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  TensorRTJSONSerializer(const std::string& symbol, const Expr& expr)
+      : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+
+    std::vector<JSONGraphNodeEntry> inputs;
+    for (const auto& arg : cn->args) {
+      auto res = VisitExpr(arg);
+      inputs.insert(inputs.end(), res.begin(), res.end());
+    }
+    auto node = std::make_shared<JSONGraphNode>(name,     /* name_ */
+                                                "kernel", /* op_type_ */
+                                                inputs, 1 /* num_outputs_ */);
+    if (name == "nn.pad") {
+      SetPadNodeAttribute(node, cn);
+    } else if (name == "strided_slice") {
+      SetStridedSliceNodeAttribute(node, cn);
+    } else {
+      SetCallNodeAttribute(node, cn);
+    }
+    // These attributes are global to the whole module.
+    SaveGlobalAttributes(node);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  void SetPadNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* pad_attr = cn->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    const int dim_h = (p.size() == 5) ? 3 : 2;
+    const int dim_w = (p.size() == 5) ? 4 : 3;
+    std::vector<std::string> padding = {std::to_string(p[dim_h][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_h][1].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    node->SetAttr("padding", padding_attr);
+  }
+
+  void SetStridedSliceNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* attrs = cn->attrs.as<StridedSliceAttrs>();
+    CHECK(attrs);
+    CHECK(attrs->begin && attrs->end && attrs->strides);
+    const bool default_strides =
+        !attrs->strides.value().defined() || attrs->strides.value().size() == 0;
+    auto ishape = backend::GetShape(cn->args[0]->checked_type());
+
+    auto process_slice_index = [](Integer x, int default_value, int dim_value) {
+      if (!x.defined()) return default_value;
+      int value = x.as<IntImmNode>()->value;
+      if (value < 0) value += dim_value;
+      return value;
+    };
+
+    std::vector<std::string> start, size, strides;
+    for (size_t i = 0; i < attrs->begin.value().size(); ++i) {
+      const int begin_value = process_slice_index(attrs->begin.value()[i], 0, ishape[i]);
+      const int end_value = process_slice_index(attrs->end.value()[i], ishape[i], ishape[i]);
+      const int stride_value = (default_strides || i >= attrs->strides.value().size() ||
+                                !attrs->strides.value()[i].defined())
+                                   ? 1
+                                   : attrs->strides.value()[i].as<IntImmNode>()->value;
+      CHECK_GT(stride_value, 0);
+      const int size_value = (end_value - begin_value + stride_value - 1) / stride_value;
+      CHECK_GE(begin_value, 0);
+      CHECK_GT(size_value, 0);
+      start.push_back(std::to_string(begin_value));
+      size.push_back(std::to_string(size_value));
+      strides.push_back(std::to_string(stride_value));
+    }
+    std::vector<dmlc::any> start_attr, size_attr, strides_attr;
+    start_attr.emplace_back(start);
+    size_attr.emplace_back(size);
+    strides_attr.emplace_back(strides);
+    node->SetAttr("start", start_attr);
+    node->SetAttr("size", size_attr);
+    node->SetAttr("strides", strides_attr);
+  }
+
+  void SaveGlobalAttributes(std::shared_ptr<JSONGraphNode> node) {
+    auto ctx = transform::PassContext::Current();
+    auto cfg = ctx->GetConfig<TensorRTCompilerConfig>("relay.ext.tensorrt.options");
+    if (!cfg.defined()) {
+      cfg = AttrsWithDefaultValues<TensorRTCompilerConfig>();
+    }
+    CHECK_EQ(cfg.value()->tensorrt_version.size(), 3);
+    std::vector<std::string> tensorrt_version = {std::to_string(cfg.value()->tensorrt_version[0]),
+                                                 std::to_string(cfg.value()->tensorrt_version[1]),
+                                                 std::to_string(cfg.value()->tensorrt_version[2])};
+    std::vector<std::string> use_implicit_batch = {std::to_string(cfg.value()->use_implicit_batch)};
+    std::vector<std::string> max_workspace_size = {std::to_string(cfg.value()->max_workspace_size)};
+    std::vector<dmlc::any> tensorrt_version_attr, use_implicit_batch_attr, max_workspace_size_attr;
+    tensorrt_version_attr.emplace_back(tensorrt_version);
+    use_implicit_batch_attr.emplace_back(use_implicit_batch);
+    max_workspace_size_attr.emplace_back(max_workspace_size);
+    node->SetAttr("tensorrt_version", tensorrt_version_attr);
+    node->SetAttr("use_implicit_batch", use_implicit_batch_attr);
+    node->SetAttr("max_workspace_size", max_workspace_size_attr);
+  }
+};
+
+/*!
+ * \brief Create a runtime module for TensorRT.
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module TensorRTCompiler(const ObjectRef& ref) {
+  CHECK(ref->IsInstance<FunctionNode>()) << "The input ref is expected to be a Relay function.";
+  Function func = Downcast<Function>(ref);
+  std::string func_name = backend::GetExtSymbol(func);
+
+  TensorRTJSONSerializer serializer(func_name, func);
+  serializer.serialize();
+  std::string graph_json = serializer.GetJSON();
+  auto param_names = serializer.GetParams();
+  const auto* pf = runtime::Registry::Get("runtime.tensorrt_runtime_create");
+  CHECK(pf != nullptr) << "Cannot find JSON runtime module to create";

Review comment:
       Improve the error message to be more TensorRT specific.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_builder.h
##########
@@ -0,0 +1,159 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_builder.h
+ * \brief The TensorRTBuilder class can be used to convert a JSONRuntime graph into a TRT engine
+ * which can be used for inference.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_BUILDER_H_
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_logger.h"
+#include "tensorrt_ops.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+/*!
+ * \brief The product of TensorRTBuilder which provides everything needed to
+ * perform inference.
+ */
+struct TrtEngineAndContext {

Review comment:
       Better to be consistent with `TensorRTBuilder`: `TensorRTEngineAndContext`.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_ops.h
##########
@@ -0,0 +1,208 @@
+/* * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/tensorrt_ops.h
+ * \brief Converters from Relay ops into TensorRT layers. Converters should
+ * inherit from TrtOpConverter and implement the Convert() method.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_OPS_H_
+
+#include <algorithm>
+#include <cmath>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "NvInfer.h"
+#include "tensorrt_utils.h"
+
+#if TRT_VERSION_GE(6, 0, 1)
+#define TRT_HAS_IMPLICIT_BATCH(params) (params->network->hasImplicitBatchDimension())
+#else
+#define TRT_HAS_IMPLICIT_BATCH(params) (true)
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+
+/*!
+ * \brief An input to a op may be either kTensor in the case of nvinfer::ITensor*
+ * or kWeight for nvinfer1::Weights.
+ */
+enum TrtInputType {
+  kTensor,
+  kWeight,
+};
+
+/*!
+ * \brief An input to a TrtOpConverter. The type of the input is either kTensor
+ * or kWeight. For kTensor, "tensor" contains the input tensor. For kWeight,
+ * "weight" contains the input weight and "weight_shape" contains the shape.
+ */
+struct TrtOpInput {
+  /*! \brief If type is kTensor, will store input tensor. */
+  nvinfer1::ITensor* tensor;
+
+  /*! \brief If type is kWeight, will store input weight. */
+  nvinfer1::Weights weight;
+
+  /*! \brief Whether the input is in tensor or weight. */
+  TrtInputType type;
+
+  /*! \brief If type is kWeight, will store weight shape. */
+  std::vector<int> weight_shape;
+
+  explicit TrtOpInput(nvinfer1::ITensor* tensor)
+      : tensor(tensor), weight({nvinfer1::DataType::kFLOAT, nullptr, 0}), type(kTensor) {}
+  TrtOpInput(nvinfer1::Weights weight, const std::vector<int>& shape)
+      : tensor(nullptr), weight(weight), type(kWeight), weight_shape(shape) {}
+};
+
+/*! \brief Parameters to convert an Op from relay to TensorRT. */
+struct AddTrtLayerParams {

Review comment:
       What does "Add" means here?

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):
+    if skip_codegen_test():
+        return
+    f, input_shapes, is_param = config
+    params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+    input_dict = {
+        k: np.random.uniform(-1, 1, v).astype(np.float32)
+        for k, v in input_shapes.items()
+        if k not in is_param
+    }
+
+    # Run TRT
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    # Run reference
+    mod = tvm.IRModule()
+    mod["main"] = f
+    with tvm.transform.PassContext(opt_level=3):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    assert len(results) == len(ref_results)
+    for i in range(len(results)):
+        res = results[i].asnumpy()
+        ref_res = ref_results[i].asnumpy()
+        assert res.shape == ref_res.shape
+        tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+
+def run_and_verify_model(model):
+    if skip_codegen_test():
+        return
+
+    def compile_and_run(i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+
+        def check_trt_used(graph):
+            import json
+
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum(
+                [
+                    1
+                    for n in graph["nodes"]
+                    if n.get("attrs", {}).get("func_name", "").startswith("tensorrt_")
+                ]
+            )
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={"data": input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod, config = tensorrt.partition_for_tensorrt(mod, params)
+            with tvm.transform.PassContext(
+                opt_level=3, config={"relay.ext.tensorrt.options": config}
+            ):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+            check_trt_used(graph)
+        else:
+            with tvm.transform.PassContext(opt_level=3):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+
+        if skip_runtime_test():
+            return
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return res
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    res = compile_and_run(i_data, input_shape, dtype, use_trt=True)
+    ref_res = compile_and_run(i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+    tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)
+
+
+def test_tensorrt_simple():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3, 1, 1)
+    zshape = (1, 1, 1, 1)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.var("y", shape=(yshape), dtype=dtype)
+    z = relay.var("z", shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_not_compatible():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 32, 14, 14)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_serialize():
+    if skip_codegen_test():
+        return
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    block = get_model("resnet18_v1", pretrained=True)
+    mod, params = relay.frontend.from_mxnet(
+        block, shape={"data": (1, 3, 224, 224)}, dtype="float32"
+    )
+    # Compile
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        lib = relay.build(mod, "cuda", params=params)
+    # Serialize
+    lib.export_library("compiled.so")
+    # Deserialize
+    loaded_lib = tvm.runtime.load_module("compiled.so")
+    # Run
+    if skip_runtime_test():
+        return
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib["default"](tvm.gpu(0)))
+    i_data = np.random.uniform(0, 1, (1, 3, 224, 224)).astype("float32")
+    for i in range(10):
+        gen_module.run(data=i_data)
+
+
+def test_conv2d():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(16, 32, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+        dilation=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    for k_shape, groups in [((16, 32, 3, 3), 1), ((32, 1, 3, 3), 32)]:
+        for padding in [(0, 0), (1, 1)]:
+            for strides in [(1, 1), (2, 2)]:
+                for dilation in [(1, 1), (2, 2)]:
+                    run_and_verify(
+                        get_graph(
+                            k_shape=k_shape,
+                            groups=groups,
+                            padding=padding,
+                            strides=strides,
+                            dilation=dilation,
+                        )
+                    )
+
+
+def test_conv2d_nhwc():
+    def get_graph(x_shape=(1, 8, 8, 32), k_shape=(3, 3, 32, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=16,
+            kernel_size=(3, 3),
+            data_layout="NHWC",
+            kernel_layout="HWIO",
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_conv2d_weights_const():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(16, 32, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+        dilation=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.const(np.ones(k_shape).astype("float32"))
+        out = relay.nn.conv2d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_conv2d_weights_transposed():
+    def get_graph(x_shape=(1, 32, 9, 9), k_shape=(3, 3, 32, 16), order=(3, 2, 0, 1)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        kernel_t = relay.transpose(kernel, order)
+        # Conv2d requires constant weights in TensorRT, so the weights should be transposed by
+        # FoldConstant.
+        out = relay.nn.conv2d(x, kernel_t, channels=k_shape[order[0]], kernel_size=(3, 3))
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_dense():
+    def get_graph(x_shape=(1, 16), k_shape=(32, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        # Dense requires constant weights in TensorRT, so the weights are transposed by us.
+        out = relay.nn.dense(x, kernel, units=k_shape[0])
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+
+
+def test_bias_add():
+    def get_graph(x_shape=(1, 16), channels=16):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        bias = relay.var("bias", shape=(channels,), dtype="float32")
+        out = relay.nn.bias_add(x, bias)
+        f = relay.Function([x, bias], out)
+        return f, {"x": x_shape, "bias": (channels,)}, ["bias"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph((1, 6, 3, 4), 6))
+
+
+def test_pool2d():
+    def get_graph(
+        op,
+        x_shape=(1, 3, 32, 32),
+        pool_size=(2, 2),
+        strides=(2, 2),
+        padding=(0, 0),
+        ceil_mode=False,
+        count_include_pad=None,
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if count_include_pad is not None:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+                count_include_pad=count_include_pad,
+            )
+        else:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for pool_size in [(2, 2), (3, 3)]:
+        for strides in [(1, 1), (2, 2)]:
+            for padding in [(0, 0), (1, 1), (0, 0, 1, 1)]:
+                for ceil_mode in [False, True]:
+                    # Skip "the padding size is larger than or equal to the filter size for exclusive-counting pooling"
+                    if pool_size == (2, 2) and padding == (0, 0, 1, 1):
+                        continue
+                    for count_include_pad in [False, True]:
+                        # Skip "inclusive-counted blended or average pooling is not supported in combination with asymmetric padding"
+                        if count_include_pad and (padding == (0, 0, 1, 1) or strides == (2, 2)):
+                            continue
+                        run_and_verify(
+                            get_graph(
+                                relay.nn.avg_pool2d,
+                                pool_size=pool_size,
+                                strides=strides,
+                                padding=padding,
+                                ceil_mode=ceil_mode,
+                                count_include_pad=count_include_pad,
+                            )
+                        )
+                    run_and_verify(
+                        get_graph(
+                            relay.nn.max_pool2d,
+                            pool_size=pool_size,
+                            strides=strides,
+                            padding=padding,
+                            ceil_mode=ceil_mode,
+                        )
+                    )
+
+
+def test_global_pool2d():
+    def get_graph(op, x_shape=(1, 3, 32, 32)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.global_max_pool2d))
+    run_and_verify(get_graph(relay.nn.global_avg_pool2d))
+
+
+def test_batch_flatten():
+    def get_graph(x_shape=(1, 3, 4, 6)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.batch_flatten(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_expand_dims():
+    def get_graph(x_shape=(1, 3), axis=1, num_newaxis=1):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.expand_dims(x, axis, num_newaxis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_squeeze():
+    def get_graph(x_shape, axis):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.squeeze(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 5, 1, 1), (2, 3)))
+    run_and_verify(get_graph((1, 3, 1), (-1,)))
+
+
+def test_concatenate():
+    def get_graph(input_shapes, axis):
+        concat_inputs = []
+        shapes_dict = {}
+        for i in range(len(input_shapes)):
+            name = "input_{}".format(i)
+            concat_inputs.append(relay.var(name, shape=(input_shapes[i]), dtype="float32"))
+            shapes_dict[name] = input_shapes[i]
+        out = relay.concatenate(concat_inputs, axis)
+        f = relay.Function(concat_inputs, out)
+        return f, shapes_dict, []
+
+    run_and_verify(get_graph([(1, 2, 6, 6), (1, 3, 6, 6)], axis=1))
+
+
+def test_conv2d_transpose():
+    def get_graph(
+        x_shape=(1, 32, 8, 8),
+        k_shape=(32, 16, 3, 3),
+        groups=1,
+        padding=(0, 0),
+        strides=(1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv2d_transpose(
+            x,
+            kernel,
+            channels=k_shape[1],
+            kernel_size=k_shape[2:4],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    for padding in [(0, 0), (1, 1)]:
+        for strides in [(1, 1), (2, 2)]:
+            run_and_verify(get_graph(padding=padding, strides=strides))
+
+
+def test_reshape():
+    def get_graph(x_shape, new_shape):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.reshape(x, new_shape)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 1, 1, 10), (-1, 10)))
+    run_and_verify(get_graph((1, 10, 2, 3), (1, -1)))
+    run_and_verify(get_graph((1, 1, 2, 3), (1, 6)))
+
+
+def test_transpose():
+    def get_graph(x_shape, order):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.transpose(x, order)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 16, 7, 7), [0, 2, 3, 1]))
+    run_and_verify(get_graph((1, 7, 7, 16), [0, 3, 1, 2]))
+
+
+def test_float_const():
+    def get_graph(x_shape=(1, 16)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        beta = relay.const(1, dtype="float32")
+        out = relay.multiply(x, beta)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_pad():
+    def get_graph(x_shape, pad_width):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.pad(x, pad_width=pad_width)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0]]))
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [1, 1], [1, 1]]))
+    run_and_verify(get_graph((1, 8, 16, 16), [[0, 0], [0, 0], [0, 1], [2, 0]]))
+    run_and_verify(get_graph((1, 8, 3, 16, 16), [[0, 0], [0, 0], [0, 0], [0, 0], [0, 0]]))
+
+
+def test_softmax():
+    def get_graph(x_shape, axis):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.softmax(x, axis=axis)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 1000), axis=1))
+    run_and_verify(get_graph((1, 1000), axis=-1))
+    run_and_verify(get_graph((1, 3, 4), axis=-2))
+    run_and_verify(get_graph((1, 3, 4), axis=1))
+
+
+def test_batch_norm():
+    def get_graph(x_shape, param_shape, axis=1, epsilon=1e-5):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        beta = relay.var("beta", shape=(param_shape), dtype="float32")
+        gamma = relay.var("gamma", shape=(param_shape), dtype="float32")
+        moving_mean = relay.var("moving_mean", shape=(param_shape), dtype="float32")
+        moving_var = relay.var("moving_var", shape=(param_shape), dtype="float32")
+        out, _, _ = relay.nn.batch_norm(
+            x,
+            gamma=gamma,
+            beta=beta,
+            moving_mean=moving_mean,
+            moving_var=moving_var,
+            axis=axis,
+            center=True,
+            scale=True,
+            epsilon=epsilon,
+        )
+        f = relay.Function([x, gamma, beta, moving_mean, moving_var], out)
+        return (
+            f,
+            {
+                "x": x_shape,
+                "beta": param_shape,
+                "gamma": param_shape,
+                "moving_mean": param_shape,
+                "moving_var": param_shape,
+            },
+            ["beta", "gamma", "moving_mean", "moving_var"],
+        )
+
+    run_and_verify(get_graph((1, 64, 56, 56), (64,)))
+    run_and_verify(get_graph((1, 56, 56, 64), (64,), axis=3, epsilon=1.001e-05))
+
+
+def test_unary():
+    def get_graph(op, x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for op in [
+        relay.nn.relu,
+        relay.sigmoid,
+        relay.tanh,
+        relay.exp,
+        relay.log,
+        relay.sqrt,
+        relay.abs,
+        relay.negative,
+        relay.sin,
+        relay.cos,
+        relay.atan,
+        relay.ceil,
+        relay.floor,
+    ]:
+        run_and_verify(get_graph(op))
+
+
+def test_clip():
+    def get_graph(x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.clip(x, a_min=-0.2, a_max=0.4)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_leaky_relu():
+    def get_graph(x_shape=(1, 8, 3, 3)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = relay.nn.leaky_relu(x, alpha=0.1)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph())
+
+
+def test_binary():
+    def get_graph(op, x_shape, y_shape, y_is_const=False):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if y_is_const:
+            y = relay.const(np.ones(y_shape).astype("float32"))
+            out = op(x, y)
+            f = relay.Function([x], out)
+            return f, {"x": x_shape}, []
+        y = relay.var("y", shape=(y_shape), dtype="float32")
+        out = op(x, y)
+        f = relay.Function([x, y], out)
+        return f, {"x": x_shape, "y": y_shape}, []
+
+    for op in [relay.add, relay.subtract, relay.multiply, relay.divide, relay.power]:
+        for y_is_const in [True, False]:
+            run_and_verify(get_graph(op, (1, 8, 3, 3), (1, 8, 3, 3), y_is_const))
+            run_and_verify(get_graph(op, (1, 8, 1, 3), (1, 8, 3, 1), y_is_const))
+            run_and_verify(get_graph(op, (1, 10), (10,), y_is_const))
+            run_and_verify(get_graph(op, (1, 1, 1, 10), (10,), y_is_const))
+            run_and_verify(get_graph(op, (1, 1, 1), (3,), y_is_const))
+
+
+def test_reduce():
+    def get_graph(op, x_shape=(1, 2, 3, 4), axis=(2, 3), keepdims=False):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x, axis=axis, keepdims=keepdims)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    for op in [relay.sum, relay.prod, relay.max, relay.min, relay.mean]:
+        for keepdims in [True, False]:
+            run_and_verify(get_graph(op, axis=(1), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(2, 3), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(1, 2), keepdims=keepdims))
+            run_and_verify(get_graph(op, axis=(1, 2, 3), keepdims=keepdims))
+
+
+def test_strided_slice():
+    def get_graph(x_shape, begin, end, strides=None):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if strides:
+            out = relay.strided_slice(
+                x,
+                relay.expr.const(begin, dtype="int32"),
+                relay.expr.const(end, dtype="int32"),
+                relay.expr.const(strides, dtype="int32"),
+            )
+        else:
+            out = relay.strided_slice(
+                x,
+                relay.expr.const(begin, dtype="int32"),
+                relay.expr.const(end, dtype="int32"),
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph((1, 3, 6, 7), [0, 0, 0, 0], [1, 1, 6, 7]))
+    run_and_verify(get_graph((1, 3, 6, 7), [0, 1, 0, 0], [1, 2, 6, 6]))
+    run_and_verify(get_graph((1, 10), [0, 0], [1, 10], [1, 2]))
+
+
+def test_adaptive_pool2d():
+    def get_graph(op, x_shape=(1, 3, 32, 32), out_size=(1, 1)):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        out = op(x, out_size)
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.adaptive_max_pool2d))
+    run_and_verify(get_graph(relay.nn.adaptive_avg_pool2d))
+
+
+def test_multiple_outputs():
+    def get_graph():
+        x = relay.var("x", shape=(1, 3), dtype="float32")
+        y = relay.var("y", shape=(1, 3), dtype="float32")
+        z = relay.add(x, y)
+        w = relay.add(z, y)
+        out = relay.Tuple((z, w))
+        f = relay.Function([x, y], out)
+        return f, {"x": (1, 3), "y": (1, 3)}, []
+
+    run_and_verify(get_graph())
+
+
+def test_conv3d():
+    def get_graph(
+        x_shape=(1, 32, 8, 8, 8),
+        k_shape=(16, 32, 3, 3, 3),
+        groups=1,
+        padding=(0, 0, 0),
+        strides=(1, 1, 1),
+        dilation=(1, 1, 1),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv3d(
+            x,
+            kernel,
+            channels=k_shape[0],
+            kernel_size=k_shape[2:],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            dilation=dilation,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph(padding=(0, 0, 0, 1, 1, 1)))
+
+
+def test_pool3d():
+    def get_graph(
+        op,
+        x_shape=(1, 3, 8, 32, 32),
+        pool_size=(2, 2, 2),
+        strides=(2, 2, 2),
+        padding=(0, 0, 0),
+        ceil_mode=False,
+        count_include_pad=None,
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        if count_include_pad is not None:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+                count_include_pad=count_include_pad,
+            )
+        else:
+            out = op(
+                x,
+                pool_size=pool_size,
+                strides=strides,
+                padding=padding,
+                ceil_mode=ceil_mode,
+            )
+        f = relay.Function([x], out)
+        return f, {"x": x_shape}, []
+
+    run_and_verify(get_graph(relay.nn.avg_pool3d))
+    run_and_verify(get_graph(relay.nn.max_pool3d))
+    run_and_verify(get_graph(relay.nn.max_pool3d, padding=(0, 0, 0, 1, 1, 1)))
+    run_and_verify(get_graph(relay.nn.max_pool3d, strides=(1, 1, 1)))
+
+
+def test_conv3d_transpose():
+    def get_graph(
+        x_shape=(1, 32, 8, 8, 8),
+        k_shape=(32, 16, 3, 3, 3),
+        groups=1,
+        padding=(0, 0, 0),
+        strides=(1, 1, 1),
+        output_padding=(0, 0, 0),
+    ):
+        x = relay.var("x", shape=(x_shape), dtype="float32")
+        kernel = relay.var("kernel", shape=(k_shape), dtype="float32")
+        out = relay.nn.conv3d_transpose(
+            x,
+            kernel,
+            channels=k_shape[1],
+            kernel_size=k_shape[2:5],
+            groups=groups,
+            padding=padding,
+            strides=strides,
+            output_padding=output_padding,
+        )
+        f = relay.Function([x, kernel], out)
+        return f, {"x": x_shape, "kernel": k_shape}, ["kernel"]
+
+    run_and_verify(get_graph())
+    run_and_verify(get_graph(strides=(2, 2, 2)))
+    run_and_verify(get_graph(strides=(2, 2, 2), output_padding=(1, 1, 1)))
+
+
+def test_alexnet():
+    run_and_verify_model("alexnet")
+
+
+def test_resnet18_v1():
+    run_and_verify_model("resnet18_v1")
+
+
+def test_resnet18_v2():
+    run_and_verify_model("resnet18_v2")
+
+
+def test_squeezenet():
+    run_and_verify_model("squeezenet1.0")
+
+
+def test_mobilenet():
+    run_and_verify_model("mobilenet0.25")
+
+
+def test_mobilenet_v2():
+    run_and_verify_model("mobilenetv2_0.25")
+
+
+def test_vgg11():
+    run_and_verify_model("vgg11")
+
+
+def test_densenet121():
+    run_and_verify_model("densenet121")

Review comment:
       I don't think we need to test that many models in the CI. Maybe one ResNet and one MobileNet would be sufficient.

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):

Review comment:
       It's unclear what this function does especially you have `run_and_verify_model`. Maybe `run_and_verify_func`?

##########
File path: src/relay/backend/contrib/tensorrt/codegen.cc
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/tensorrt/codegen.cc
+ * \brief Implementation of the TensorRT JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../utils.h"
+#include "../codegen_json/codegen_json.h"
+
+#if TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#endif
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*! \brief Attributes to store the compiler options for TensorRT. */
+struct TensorRTCompilerConfigNode : public tvm::AttrsNode<TensorRTCompilerConfigNode> {
+  Array<Integer> tensorrt_version;
+  bool use_implicit_batch;
+  size_t max_workspace_size;
+  bool remove_no_mac_subgraphs;
+
+  TVM_DECLARE_ATTRS(TensorRTCompilerConfigNode, "ext.attrs.TensorRTCompilerConfigNode") {
+    TVM_ATTR_FIELD(tensorrt_version)
+        .describe("TensorRT version as (major, minor, patch).")
+        .set_default(Array<Integer>({6, 0, 1}));
+    TVM_ATTR_FIELD(use_implicit_batch).set_default(true);
+    TVM_ATTR_FIELD(max_workspace_size).set_default(size_t(1) << 30);
+    TVM_ATTR_FIELD(remove_no_mac_subgraphs).set_default(false);
+  }
+};
+
+class TensorRTCompilerConfig : public Attrs {
+ public:
+  TVM_DEFINE_NOTNULLABLE_OBJECT_REF_METHODS(TensorRTCompilerConfig, Attrs,
+                                            TensorRTCompilerConfigNode);
+};
+
+TVM_REGISTER_NODE_TYPE(TensorRTCompilerConfigNode);
+TVM_REGISTER_PASS_CONFIG_OPTION("relay.ext.tensorrt.options", TensorRTCompilerConfig);
+
+/*!
+ * \brief Generates an TensorRTModule from a relay expression by serializing the expression to a
+ * json representation. TensorRT is not required here because use of TensorRT APIs is deferred until
+ * runtime.
+ */
+class TensorRTJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  TensorRTJSONSerializer(const std::string& symbol, const Expr& expr)
+      : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+
+    std::vector<JSONGraphNodeEntry> inputs;
+    for (const auto& arg : cn->args) {
+      auto res = VisitExpr(arg);
+      inputs.insert(inputs.end(), res.begin(), res.end());
+    }
+    auto node = std::make_shared<JSONGraphNode>(name,     /* name_ */
+                                                "kernel", /* op_type_ */
+                                                inputs, 1 /* num_outputs_ */);
+    if (name == "nn.pad") {
+      SetPadNodeAttribute(node, cn);
+    } else if (name == "strided_slice") {
+      SetStridedSliceNodeAttribute(node, cn);
+    } else {
+      SetCallNodeAttribute(node, cn);
+    }
+    // These attributes are global to the whole module.
+    SaveGlobalAttributes(node);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  void SetPadNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* pad_attr = cn->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    const int dim_h = (p.size() == 5) ? 3 : 2;
+    const int dim_w = (p.size() == 5) ? 4 : 3;
+    std::vector<std::string> padding = {std::to_string(p[dim_h][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][0].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_h][1].as<IntImmNode>()->value),
+                                        std::to_string(p[dim_w][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    node->SetAttr("padding", padding_attr);
+  }
+
+  void SetStridedSliceNodeAttribute(std::shared_ptr<JSONGraphNode> node, const CallNode* cn) {
+    const auto* attrs = cn->attrs.as<StridedSliceAttrs>();
+    CHECK(attrs);
+    CHECK(attrs->begin && attrs->end && attrs->strides);

Review comment:
       We can merge these two checks and provide a proper error message.

##########
File path: src/runtime/contrib/tensorrt/tensorrt_utils.h
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file runtime/contrib/tensorrt/utils.h
+ * \brief Helper functions used by TensorRTBuilder or TrtOpConverters.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+#define TVM_RUNTIME_CONTRIB_TENSORRT_TENSORRT_UTILS_H_
+
+#include <string>
+#include <vector>
+
+#include "NvInfer.h"
+
+// There is a conflict between cpplint and clang-format-10.
+// clang-format off
+#define TRT_VERSION_GE(major, minor, patch)                                                    \
+  ((NV_TENSORRT_MAJOR > major) || (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR > minor) || \
+  (NV_TENSORRT_MAJOR == major && NV_TENSORRT_MINOR == minor && NV_TENSORRT_PATCH >= patch))
+// clang-format on
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+/*!
+ * \brief Helper function to convert an vector to TRT Dims.
+ * \param vec Vector.
+ * \return TRT Dims.
+ */
+template <typename T>
+inline nvinfer1::Dims VectorToTrtDims(const std::vector<T>& vec) {
+  nvinfer1::Dims dims;
+  // Dims(nbDims=0, d[0]=1) is used to represent a scalar in TRT.
+  dims.d[0] = 1;
+  dims.nbDims = vec.size();
+  for (size_t i = 0; i < vec.size(); ++i) {
+    dims.d[i] = vec[i];
+  }
+  return dims;
+}
+
+/*!
+ * \brief Helper function to convert TRT Dims to vector.
+ * \param vec TRT Dims.
+ * \return Vector.
+ */
+inline std::vector<int> TrtDimsToVector(const nvinfer1::Dims& dims) {
+  return std::vector<int>(dims.d, dims.d + dims.nbDims);
+}
+
+/*!
+ * \brief Helper function to convert vector to string.
+ * \param vec Vector.
+ * \return Vector as a string.
+ */
+template <typename T>
+inline std::string DebugString(const std::vector<T>& vec) {

Review comment:
       @zhiics do we have existing helper functions to achieve the same goal?

##########
File path: tests/python/contrib/test_tensorrt.py
##########
@@ -0,0 +1,896 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import numpy as np
+import time
+import pytest
+
+import tvm
+import tvm.relay.testing
+from tvm import relay
+from tvm.relay.op.contrib import tensorrt
+from tvm.contrib import graph_runtime
+
+
+def skip_codegen_test():
+    """Skip test if TensorRT and CUDA codegen are not present"""
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tvm.get_global_func("relay.ext.tensorrt", True):
+        print("Skip because TensorRT codegen is not available.")
+        return True
+    return False
+
+
+def skip_runtime_test():
+    if not tvm.runtime.enabled("cuda") or not tvm.gpu(0).exist:
+        print("Skip because CUDA is not enabled.")
+        return True
+    if not tensorrt.is_tensorrt_runtime_enabled():
+        print("Skip because TensorRT runtime is not available.")
+        return True
+    return False
+
+
+def run_and_verify(config):
+    if skip_codegen_test():
+        return
+    f, input_shapes, is_param = config
+    params = {x: np.random.uniform(-1, 1, input_shapes[x]).astype(np.float32) for x in is_param}
+    input_dict = {
+        k: np.random.uniform(-1, 1, v).astype(np.float32)
+        for k, v in input_shapes.items()
+        if k not in is_param
+    }
+
+    # Run TRT
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    # Run reference
+    mod = tvm.IRModule()
+    mod["main"] = f
+    with tvm.transform.PassContext(opt_level=3):
+        graph, lib, graph_params = relay.build(mod, "cuda", params=params)
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    mod.set_input(**graph_params)
+    mod.run(**input_dict)
+    ref_results = [mod.get_output(i) for i in range(mod.get_num_outputs())]
+
+    assert len(results) == len(ref_results)
+    for i in range(len(results)):
+        res = results[i].asnumpy()
+        ref_res = ref_results[i].asnumpy()
+        assert res.shape == ref_res.shape
+        tvm.testing.assert_allclose(res, ref_res, rtol=1e-3, atol=1e-3)
+
+
+def run_and_verify_model(model):
+    if skip_codegen_test():
+        return
+
+    def compile_and_run(i_data, input_shape, dtype, use_trt=True, num_iteration=1):
+        import mxnet as mx
+        from mxnet.gluon.model_zoo.vision import get_model
+
+        def check_trt_used(graph):
+            import json
+
+            graph = json.loads(graph)
+            num_trt_subgraphs = sum(
+                [
+                    1
+                    for n in graph["nodes"]
+                    if n.get("attrs", {}).get("func_name", "").startswith("tensorrt_")
+                ]
+            )
+            assert num_trt_subgraphs >= 1
+
+        block = get_model(model, pretrained=True)
+        mod, params = relay.frontend.from_mxnet(block, shape={"data": input_shape}, dtype=dtype)
+
+        if use_trt:
+            mod, config = tensorrt.partition_for_tensorrt(mod, params)
+            with tvm.transform.PassContext(
+                opt_level=3, config={"relay.ext.tensorrt.options": config}
+            ):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+            check_trt_used(graph)
+        else:
+            with tvm.transform.PassContext(opt_level=3):
+                graph, lib, params = relay.build(mod, "cuda", params=params)
+
+        if skip_runtime_test():
+            return
+        mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+        mod.set_input(**params)
+        # Warmup
+        for i in range(10):
+            mod.run(data=i_data)
+        # Time
+        times = []
+        for i in range(num_iteration):
+            start_time = time.time()
+            mod.run(data=i_data)
+            res = mod.get_output(0)
+            times.append(time.time() - start_time)
+        latency = 1000.0 * np.mean(times)
+        print(model, latency)
+        return res
+
+    dtype = "float32"
+    input_shape = (1, 3, 224, 224)
+    i_data = np.random.uniform(-1, 1, input_shape).astype(dtype)
+    res = compile_and_run(i_data, input_shape, dtype, use_trt=True)
+    ref_res = compile_and_run(i_data, input_shape, dtype, use_trt=False, num_iteration=1)
+    tvm.testing.assert_allclose(res.asnumpy(), ref_res.asnumpy(), rtol=1e-3, atol=1e-3)
+
+
+def test_tensorrt_simple():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 3, 2, 2)
+    yshape = (1, 3, 1, 1)
+    zshape = (1, 1, 1, 1)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.var("y", shape=(yshape), dtype=dtype)
+    z = relay.var("z", shape=(zshape), dtype=dtype)
+    w = z * (x + y)
+    out = relay.nn.relu(w)
+    f = relay.Function([x, y, z], out)
+
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    y_data = np.random.uniform(-1, 1, yshape).astype(dtype)
+    z_data = np.random.uniform(-1, 1, zshape).astype(dtype)
+    mod.run(x=x_data, y=y_data, z=z_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_not_compatible():
+    if skip_codegen_test():
+        return
+    dtype = "float32"
+    xshape = (1, 32, 14, 14)
+    x = relay.var("x", shape=(xshape), dtype=dtype)
+    y = relay.add(x, x)
+    z = relay.erf(y)
+    out = relay.nn.relu(z)
+    f = relay.Function([x], out)
+    mod = tvm.IRModule()
+    mod["main"] = f
+    mod, config = tensorrt.partition_for_tensorrt(mod)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        graph, lib, params = relay.build(mod, "cuda")
+    if skip_runtime_test():
+        return
+    mod = graph_runtime.create(graph, lib, ctx=tvm.gpu(0))
+    x_data = np.random.uniform(-1, 1, xshape).astype(dtype)
+    mod.run(x=x_data)
+    results = [mod.get_output(i).asnumpy() for i in range(mod.get_num_outputs())]
+
+
+def test_tensorrt_serialize():
+    if skip_codegen_test():
+        return
+    import mxnet
+    from mxnet.gluon.model_zoo.vision import get_model
+
+    block = get_model("resnet18_v1", pretrained=True)
+    mod, params = relay.frontend.from_mxnet(
+        block, shape={"data": (1, 3, 224, 224)}, dtype="float32"
+    )
+    # Compile
+    mod, config = tensorrt.partition_for_tensorrt(mod, params)
+    with tvm.transform.PassContext(opt_level=3, config={"relay.ext.tensorrt.options": config}):
+        lib = relay.build(mod, "cuda", params=params)
+    # Serialize
+    lib.export_library("compiled.so")
+    # Deserialize
+    loaded_lib = tvm.runtime.load_module("compiled.so")
+    # Run
+    if skip_runtime_test():
+        return
+    gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib["default"](tvm.gpu(0)))
+    i_data = np.random.uniform(0, 1, (1, 3, 224, 224)).astype("float32")
+    for i in range(10):
+        gen_module.run(data=i_data)

Review comment:
       Why you need to run 10 times?

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names),
+        use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {

Review comment:
       Could you remind me what other possible types here?

##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       This is an interesting discussion. I realized that this is more like a serialization for platform-dependent TensorRT engines. If it's not possible to build and serialize the engine during the compilation (or cross-compilation) even we have built the TVM with TensorRT runtime, then this is probably inevitable; otherwise we may build the engine and serialize the bit-stream along with other artifacts in `SaveToBinary`.
   
   If the serialization here is inevitable, which I believe in it because users may not have TensorRT during compilation, then the next question is whether we can update the ".so" file with the serialized engine here instead of creating a separate file. In other words, the .so file may or may not contain a serialized engine, but if it has, we don't need to build it again.
   
   
   
   




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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707363920


   > @trevor-m I don't see anything weird with your build rules, but I wonder if changing the cmake config affected something. do you have >1 CI failure showing the segfault, or can you reproduce this locally?
   > 
   > we have seen this before sporadically but don't know what causes it, and it's usually pretty hard to reproduce
   
   Yes, every CI run with USE_TENSORRT_CODEGEN ON got the segfault (there was at least 10 runs).
   I was able to reproduce this consistently locally now by replicating the same steps used by the CI (using docker image). It's the `apps/bundle_deploy/build_model.py` script which is segfaulting. I ran it using gdb inside the container.
   
   From `gdb -ex r --args python3 build_model.py -o build --test`.
   ```
   Thread 1 "python3" received signal SIGSEGV, Segmentation fault.
   __GI___libc_free (mem=0x6) at malloc.c:2958
   2958    malloc.c: No such file or directory.
   (gdb) bt
   #0  __GI___libc_free (mem=0x6) at malloc.c:2958
   #1  0x00007fffde4937f4 in dmlc::parameter::FieldAccessEntry::~FieldAccessEntry() () from /workspace/build/libtvm.so
   #2  0x00007fff9702a4af in dmlc::parameter::FieldEntry<std::string>::~FieldEntry() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #3  0x00007fff97037267 in dmlc::parameter::ParamManager::~ParamManager() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #4  0x00007ffff6cd7008 in __run_exit_handlers (status=0, listp=0x7ffff70615f8 <__exit_funcs>, run_list_atexit=run_list_atexit@entry=true) at exit.c:82
   #5  0x00007ffff6cd7055 in __GI_exit (status=<optimized out>) at exit.c:104
   #6  0x00007ffff6cbd847 in __libc_start_main (main=0x4d1cb0 <main>, argc=5, argv=0x7fffffffe858, init=<optimized out>, fini=<optimized out>, rtld_fini=<optimized out>, stack_end=0x7fffffffe848) at ../csu/libc-start.c:325
   #7  0x00000000005e8569 in _start ()
   ```


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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   @tqchen I just checked the TensorRT docker image and found that it is released and hosted by Nvidia on its own hub (https://ngc.nvidia.com/catalog/containers/nvidia:tensorrt/tags) and it is not available on Docker hub. I am not sure if license would be an issue if TVM CI (on Docker hub) uses that as the base image.
   
   Also from @trevor-m, the TensorRT image includes CUDA/CuDNN, but we need to check their versions by ourselves. For example, 20.08 has CUDA 11.


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

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



[GitHub] [incubator-tvm] trevor-m commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707923371


   > I think we can enable the test and merge after #6679 is landed since its pretty close already?
   
   Agreed, makes sense. I'll renable test once #6679 is merged.


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

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



[GitHub] [incubator-tvm] trevor-m edited a comment on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m edited a comment on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707363920


   > @trevor-m I don't see anything weird with your build rules, but I wonder if changing the cmake config affected something. do you have >1 CI failure showing the segfault, or can you reproduce this locally?
   > 
   > we have seen this before sporadically but don't know what causes it, and it's usually pretty hard to reproduce
   
   Yes, every CI run with USE_TENSORRT_CODEGEN ON got the segfault (there was at least 10 runs).
   I was able to reproduce this consistently locally now by replicating the same steps used by the CI (using docker image). It's the `apps/bundle_deploy/build_model.py` script which is segfaulting. I ran it using gdb inside the container.
   
   From `gdb -ex r --args python3 build_model.py -o build --test`.
   ```
   Thread 1 "python3" received signal SIGSEGV, Segmentation fault.
   __GI___libc_free (mem=0x6) at malloc.c:2958
   2958    malloc.c: No such file or directory.
   (gdb) bt
   #0  __GI___libc_free (mem=0x6) at malloc.c:2958
   #1  0x00007fffde4937f4 in dmlc::parameter::FieldAccessEntry::~FieldAccessEntry() () from /workspace/build/libtvm.so
   #2  0x00007fff9702a4af in dmlc::parameter::FieldEntry<std::string>::~FieldEntry() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #3  0x00007fff97037267 in dmlc::parameter::ParamManager::~ParamManager() () from /usr/local/lib/python3.6/dist-packages/xgboost/./lib/libxgboost.so
   #4  0x00007ffff6cd7008 in __run_exit_handlers (status=0, listp=0x7ffff70615f8 <__exit_funcs>, run_list_atexit=run_list_atexit@entry=true) at exit.c:82
   #5  0x00007ffff6cd7055 in __GI_exit (status=<optimized out>) at exit.c:104
   #6  0x00007ffff6cbd847 in __libc_start_main (main=0x4d1cb0 <main>, argc=5, argv=0x7fffffffe858, init=<optimized out>, fini=<optimized out>, rtld_fini=<optimized out>, stack_end=0x7fffffffe848) at ../csu/libc-start.c:325
   #7  0x00000000005e8569 in _start ()
   ```
   
   I found that the issue is caused by the xgboost version in the CI docker is too old and has an incompatible dmlc-core commit. The container had xgboost 1.0.2. I upgraded to 1.2 and now the segfault is fixed.


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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

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


   > Hmm, this seems like it would make the job of the `PruneTensorRTSubgraph` pass much more difficult. `PartitionGraph` already takes care of collecting the inputs and outputs of a subgraph and additional processing such as making sure there are no duplicate outputs. If `PruneTesnorRTCompilerRegion` was before `PartitionGraph`, it would have to duplicate a lot of that work. The idea of the pruning pass is that we should present each backend with the final subgraph exactly as it would be when it is passed to the codegen and the backend should decide if it is valid or not. Are you concerned about the overhead of partitioning a subgraph which would be later discarded?
   >
   > Btw just for referece, here is the general implementation of PruneSubgraph that I originally implemented: [trevor-m@06015a4](https://github.com/trevor-m/tvm/commit/06015a4617cfaad56adcaa0c71b485d6bd711128)
   > 
   
   My main concern was that it would be tedious to have a `partition_graph -> revert_some_partitions` flow. Also in this case, your post-processing pass depends on the partition pass and may fail along with the change of the partition pass. If this requirement is important, I'd even prefer to add post-processing feature to the partition pass that allows you to provide a packed function to check if a partitioned function is valid.
   
   On the other hand, in order to not block this PR  for too long, we can maybe follow the current flow first, and discuss a plan of refactoring the partition pass to better support this requirement.
   
   @zhiics do you have any suggestion?
   
   > I have already created an API to retrieve the TRT version if TVM is compiled with the TRT runtime enabled. However, one of our use cases is to use TVM on a CPU-only instance to cross-compile models. For that use case, we want to be able to target compilation for different TRT versions - this affects the partitioning rules mostly. I don't think having to rebuild TVM for each target version will be a good solution.
   > 
   > Is it possible for my annotation functions to access the pass context and therefore a TRT config that I will be adding as @masahi suggested? I don't see any other python code accessing the PassContext though...
   
   Looks like `GetConfig` does not expose to the Python side.


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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r489720567



##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       Building the TensorRT engine which is done on the first inference can be very slow. On edge devices it can even take up to an hour. NVIDIA provides an API to serialize/load the built TRT engine after it is built to avoid repeating this slow process.
   
   This serialization method is separate from the LoadFrom/SaveToBinary and is there to expose TRT's engine serialization/loading API to the user so they won't have to rebuild the engine every time they load the model.
   
   There is some more info here: https://neo-ai-dlr.readthedocs.io/en/latest/tensorrt.html#caching-tensorrt-engines




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

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



[GitHub] [incubator-tvm] trevor-m commented on a change in pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
trevor-m commented on a change in pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#discussion_r497068599



##########
File path: src/runtime/contrib/tensorrt/tensorrt_runtime.cc
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \file src/runtime/contrib/tensorrt/tensorrt_runtime.cc
+ * \brief JSON runtime implementation for TensorRT.
+ */
+
+#include <dmlc/parameter.h>
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+#include "NvInfer.h"
+#include "tensorrt_builder.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+class TensorRTRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The TensorRT runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit TensorRTRuntime(const std::string& symbol_name, const std::string& graph_json,
+                           const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names), use_implicit_batch_(true),
+        max_workspace_size_(size_t(1) << 30) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "tensorrt"; }
+
+  /*!
+   * \brief Initialize runtime. Create TensorRT layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    LoadGlobalAttributes();
+    if (GetCachedEnginesFromDisk()) return;
+    SetupConstants(consts);
+    BuildEngine();
+    CacheEngineToDisk();
+  }
+
+  void LoadGlobalAttributes() {
+    // These settings are global to the entire subgraph. Codegen will add them as attributes to all
+    // op nodes. Read from first one.
+    for (size_t i = 0; i < nodes_.size(); ++i) {
+      if (nodes_[i].HasAttr("use_implicit_batch") && nodes_[i].HasAttr("max_workspace_size")) {
+        use_implicit_batch_ =
+            std::stoi(nodes_[i].GetAttr<std::vector<std::string>>("use_implicit_batch")[0]);
+        // Allow max_workspace_size to be overridden at runtime.
+        size_t runtime_max_workspace_size =
+            dmlc::GetEnv("TVM_TENSORRT_MAX_WORKSPACE_SIZE", size_t(0));
+        if (runtime_max_workspace_size != 0) {
+          max_workspace_size_ = runtime_max_workspace_size;
+        } else {
+          max_workspace_size_ =
+              std::stoul(nodes_[i].GetAttr<std::vector<std::string>>("max_workspace_size")[0]);
+        }
+        return;
+      }
+    }
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_TENSORRT
+  /*! \brief Run inference using built engine. */
+  void Run() override {
+    auto& engine_and_context = trt_engine_cache_.at(symbol_name_);
+    auto engine = engine_and_context.engine;
+    auto context = engine_and_context.context;
+    std::vector<void*> bindings(engine->getNbBindings(), nullptr);
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      if (nodes_[nid].GetOpType() == "input") {
+        for (size_t j = 0; j < nodes_[nid].GetOpShape().size(); ++j) {
+          uint32_t eid = EntryID(nid, j);
+          const std::string name = nodes_[nid].GetOpName() + "_" + std::to_string(j);
+          int binding_index = engine->getBindingIndex(name.c_str());
+          CHECK_NE(binding_index, -1);
+          bindings[binding_index] = data_entry_[eid]->data;
+        }
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      const std::string& name = engine_and_context.outputs[i];
+      int binding_index = engine->getBindingIndex(name.c_str());
+      CHECK_NE(binding_index, -1);
+      bindings[binding_index] = data_entry_[eid]->data;
+    }
+
+#if TRT_VERSION_GE(6, 0, 1)
+    if (use_implicit_batch_) {
+      CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+    } else {
+      CHECK(context->executeV2(bindings.data())) << "Running TensorRT failed.";
+    }
+#else
+    CHECK(context->execute(batch_size_, bindings.data())) << "Running TensorRT failed.";
+#endif
+  }
+
+ private:
+  /*!
+   * \brief Build TensorRT engine from JSON representation.
+   */
+  void BuildEngine() {
+    LOG(INFO) << "Building new TensorRT engine for subgraph " << symbol_name_;
+    const bool use_fp16 = dmlc::GetEnv("TVM_TENSORRT_USE_FP16", false);
+    batch_size_ = GetBatchSize();
+    TensorRTBuilder builder(&logger_, max_workspace_size_, use_implicit_batch_, use_fp16,
+                            batch_size_);
+
+    // Add inputs and constants.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      std::string name = node.GetOpName();
+      if (node.GetOpType() == "input") {
+        builder.AddInput(nid, node);
+      } else {
+        CHECK_EQ(node.GetOpType(), "const");
+        uint32_t eid = EntryID(nid, 0);
+        builder.AddConstant(nid, data_entry_[eid]);
+      }
+    }
+
+    // Add layers.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() != "kernel") continue;
+      builder.AddLayer(nid, node);
+    }
+
+    // Add outputs.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      builder.AddOutput(outputs_[i]);
+    }
+
+    // Build engine.
+    trt_engine_cache_[symbol_name_] = builder.BuildEngine();
+    LOG(INFO) << "Finished building TensorRT engine for subgraph " << symbol_name_;
+  }
+
+  /*! \brief If TVM_TENSORRT_CACHE_DIR is set, will check that directory for
+   * already built TRT engines and load into trt_engine_cache_ so they don't
+   * have to be built at first inference.
+   */
+  bool GetCachedEnginesFromDisk() {

Review comment:
       Thanks @comaniac that is correct. The engine is platform-dependent so it is not possible to create it during the compilation, it must be done at runtime.
   
   It think it is an interesting idea to update the `.so` with the built engine. I think the TVM runtime doesn't contain the necessary components to be able to serialize to .so. It could also introduce some weird behavior (you run a model on one NVIDIA device, it stores the built engine in the .so, then you take the model and try to run it on a different NVIDIA device and it wouldn't work).
   
   This extra serialization is not required to use TRT which is why it is only exposed via an optional environment variable. It is useful for edge devices however where building the TRT engine can take up to an hour.




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

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



[GitHub] [incubator-tvm] areusch commented on pull request #6395: [BYOC][TensorRT] TensorRT BYOC integration

Posted by GitBox <gi...@apache.org>.
areusch commented on pull request #6395:
URL: https://github.com/apache/incubator-tvm/pull/6395#issuecomment-707375037


   @trevor-m I was able to reproduce this as well on the ci-gpu image. it looks similar to https://discuss.xgboost.ai/t/segfault-during-code-cleanup/1365/6
   
   i don't think it's related to `apps/bundle_deploy` other than that maybe that's the first thing that runs through a full TVM python session and then quits? I don't know why it happens with USE_TENSORRT on, though.
   
   @tqchen any ideas?


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

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