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/06/24 15:54:25 UTC

[GitHub] [incubator-tvm] lhutton1 opened a new pull request #5915: [BYOC][Contrib] Arm Compute Library integration

lhutton1 opened a new pull request #5915:
URL: https://github.com/apache/incubator-tvm/pull/5915


   Arm Compute Library (ACL) integration using the BYOC infrastructure. This will enable offloading select operators from a relay graph to ACL so we can achieve faster inference times on Arm CPU's due to hand crafted optimized routines. The PR adds initial support for offloading FP32 conv2d, maxpool2d and reshape to ACL. ACL codegen is used to generate a JSON representation of an operator or 'ACL layer', the ACL runtime then uses this representation to construct a layer, cache it and create a packed
   function to for the graph runtime to call into.
   
   RFC here: https://discuss.tvm.ai/t/rfc-byoc-arm-compute-library-integration/7082
   
   Change-Id: If756dcea787ea346b1508e9a191b7eed7bd02b7f
   


----------------------------------------------------------------
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] FrozenGene commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""

Review comment:
       `ACL` -> `ARM Compute Library`

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        json, lib, params = relay.build(module, target=target)

Review comment:
       As we have supported model based runtime, `relay.build` now returns `lib`, which contains `json` / `params`. Here, we could use the new behavior, just return `lib = relay.build(module, target=target)` 

##########
File path: tests/python/contrib/test_arm_compute_lib/infrastructure.py
##########
@@ -0,0 +1,167 @@
+# 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.
+from itertools import zip_longest, combinations
+import json
+
+import tvm
+from tvm import relay
+from tvm import rpc
+from tvm.contrib import graph_runtime
+from tvm.relay.op.contrib import arm_compute_lib
+from tvm.contrib import util
+
+
+class Device:
+    """Adjust the following settings to connect to and use a remote device for tests."""
+    use_remote = False
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    # Enable cross compilation when connecting a remote device from a non-arm platform.
+    cross_compile = None
+    # cross_compile = "aarch64-linux-gnu-g++"
+
+    def __init__(self):
+        """Keep remote device for lifetime of object."""
+        self.device = self._get_remote()
+
+    @classmethod
+    def _get_remote(cls):
+        """Get a remote (or local) device to use for testing."""
+        if cls.use_remote:
+            # Here you may adjust settings to run the ACL unit tests via a remote
+            # device using the RPC mechanism. Use this in the case you want to compile
+            # an ACL module on a different machine to what you run the module on i.e.
+            # x86 -> AArch64.
+            #
+            # Use the following to connect directly to a remote device:
+            # device = rpc.connect(
+            #     hostname="0.0.0.0",
+            #     port=9090)
+            #
+            # Or connect via a tracker:
+            # device = tvm.autotvm.measure.request_remote(
+            #     host="0.0.0.0",
+            #     port=9090,
+            #     device_key="device_key",
+            #     timeout=1000)
+            #
+            # return device
+            raise NotImplementedError(
+                "Please adjust these settings to connect to your remote device.")
+        else:
+            device = rpc.LocalSession()
+            return device
+
+
+def skip_runtime_test():
+    """Skip test if it requires the runtime and it's not present."""
+    # ACL codegen not present.
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+    # Remote device is in use or ACL runtime not present
+    if not Device.use_remote and not arm_compute_lib.is_arm_compute_runtime_enabled():
+        print("Skip because runtime isn't present or a remote device isn't being used.")
+        return True
+
+
+def skip_codegen_test():
+    """Skip test if it requires the ACL codegen and it's not present."""
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+
+def build_module(mod, target, params=None, enable_acl=True):
+    """Build module with option to build for ACL."""
+    if isinstance(mod, tvm.relay.expr.Call):
+        mod = tvm.IRModule.from_expr(mod)
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        if enable_acl:
+            mod = arm_compute_lib.partition_for_arm_compute_lib(mod, params)
+        relay.backend.compile_engine.get().clear()
+        return relay.build(mod, target=target, params=params)
+
+
+def build_and_run(mod, inputs, outputs, params, device, enable_acl=True, no_runs=1):
+    """Build and run the relay module."""
+    graph, lib, params = build_module(mod, device.target, params, enable_acl)
+    lib = update_lib(lib, device.device, device.cross_compile)
+    gen_module = graph_runtime.create(graph, lib, ctx=device.device.cpu(0))
+    gen_module.set_input(**inputs)
+    gen_module.set_input(**params)
+    for _ in range(no_runs):
+        gen_module.run()
+    out = [gen_module.get_output(i) for i in range(outputs)]
+    return out
+
+
+def update_lib(lib, device, cross_compile):
+    """Export the library to the remote/local device."""
+    lib_name = "mod.so"
+    temp = util.tempdir()
+    lib_path = temp.relpath(lib_name)
+    if cross_compile:
+        lib.export_library(lib_path, cc=cross_compile)
+    else:
+        lib.export_library(lib_path)
+    device.upload(lib_path)
+    lib = device.load_module(lib_name)
+    return lib
+
+
+def verify(answers, atol, rtol):
+    """Compare the array of answers. Each entry is a list of outputs."""
+    if len(answers) < 2:
+        raise RuntimeError(
+            f"No results to compare: expected at least two, found {len(answers)}")
+    for answer in zip_longest(*answers):
+        for outs in combinations(answer, 2):
+            tvm.testing.assert_allclose(
+               outs[0].asnumpy(), outs[1].asnumpy(), rtol=rtol, atol=atol)
+
+
+def extract_acl_modules(module):
+    """Get the ACL module(s) from llvm module."""
+    return list(filter(lambda mod: mod.type_key == "arm_compute_lib",
+                       module.imported_modules))
+
+
+def verify_codegen(module, known_good_codegen, num_acl_modules,
+                   target="llvm -mtriple=aarch64-linux-gnu -mattr=+neon"):
+    """Check acl codegen against a known good output."""
+    _, module, _ = build_module(module, target)

Review comment:
       ```python
   module = build_module(module, target)
   ```

##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_enabled():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    return tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.MergeComposite(arm_compute_lib_pattern_table()),
+                                    transform.AnnotateTarget('arm_compute_lib'),
+                                    transform.PartitionGraph()])
+
+    return seq(mod)
+
+
+@register_pattern_table("arm_compute_lib")
+def arm_compute_lib_pattern_table():
+    """Get the ACL pattern table."""
+
+    def conv_pattern():
+        """Create a convolution pattern.
+
+        Returns
+        -------
+        pattern : dataflow_pattern.AltPattern
+            Denotes the convolution pattern.
+        """
+        pattern = is_op('nn.pad')(wildcard()) | wildcard()
+        pattern = is_op('nn.conv2d')(pattern, is_constant())
+        pattern = pattern.optional(lambda x: is_op('nn.bias_add')(x, is_constant()))
+        pattern = pattern.optional(is_op('nn.relu'))

Review comment:
       Curious question, does ACL support to fuse `relu6`? which is implemented by op `clip` in TVM.

##########
File path: tests/python/contrib/test_arm_compute_lib/infrastructure.py
##########
@@ -0,0 +1,167 @@
+# 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.
+from itertools import zip_longest, combinations
+import json
+
+import tvm
+from tvm import relay
+from tvm import rpc
+from tvm.contrib import graph_runtime
+from tvm.relay.op.contrib import arm_compute_lib
+from tvm.contrib import util
+
+
+class Device:
+    """Adjust the following settings to connect to and use a remote device for tests."""
+    use_remote = False
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    # Enable cross compilation when connecting a remote device from a non-arm platform.
+    cross_compile = None
+    # cross_compile = "aarch64-linux-gnu-g++"
+
+    def __init__(self):
+        """Keep remote device for lifetime of object."""
+        self.device = self._get_remote()
+
+    @classmethod
+    def _get_remote(cls):
+        """Get a remote (or local) device to use for testing."""
+        if cls.use_remote:
+            # Here you may adjust settings to run the ACL unit tests via a remote
+            # device using the RPC mechanism. Use this in the case you want to compile
+            # an ACL module on a different machine to what you run the module on i.e.
+            # x86 -> AArch64.
+            #
+            # Use the following to connect directly to a remote device:
+            # device = rpc.connect(
+            #     hostname="0.0.0.0",
+            #     port=9090)
+            #
+            # Or connect via a tracker:
+            # device = tvm.autotvm.measure.request_remote(
+            #     host="0.0.0.0",
+            #     port=9090,
+            #     device_key="device_key",
+            #     timeout=1000)
+            #
+            # return device
+            raise NotImplementedError(
+                "Please adjust these settings to connect to your remote device.")
+        else:
+            device = rpc.LocalSession()
+            return device
+
+
+def skip_runtime_test():
+    """Skip test if it requires the runtime and it's not present."""
+    # ACL codegen not present.
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+    # Remote device is in use or ACL runtime not present
+    if not Device.use_remote and not arm_compute_lib.is_arm_compute_runtime_enabled():
+        print("Skip because runtime isn't present or a remote device isn't being used.")
+        return True
+
+
+def skip_codegen_test():
+    """Skip test if it requires the ACL codegen and it's not present."""
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+
+def build_module(mod, target, params=None, enable_acl=True):
+    """Build module with option to build for ACL."""
+    if isinstance(mod, tvm.relay.expr.Call):
+        mod = tvm.IRModule.from_expr(mod)
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        if enable_acl:
+            mod = arm_compute_lib.partition_for_arm_compute_lib(mod, params)
+        relay.backend.compile_engine.get().clear()
+        return relay.build(mod, target=target, params=params)
+
+
+def build_and_run(mod, inputs, outputs, params, device, enable_acl=True, no_runs=1):
+    """Build and run the relay module."""
+    graph, lib, params = build_module(mod, device.target, params, enable_acl)
+    lib = update_lib(lib, device.device, device.cross_compile)
+    gen_module = graph_runtime.create(graph, lib, ctx=device.device.cpu(0))

Review comment:
       ```python
   gen_module = graph_runtime.GraphModule(lib['default'](device.device.cpu(0)))
   ```

##########
File path: tests/python/contrib/test_arm_compute_lib/infrastructure.py
##########
@@ -0,0 +1,167 @@
+# 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.
+from itertools import zip_longest, combinations
+import json
+
+import tvm
+from tvm import relay
+from tvm import rpc
+from tvm.contrib import graph_runtime
+from tvm.relay.op.contrib import arm_compute_lib
+from tvm.contrib import util
+
+
+class Device:
+    """Adjust the following settings to connect to and use a remote device for tests."""
+    use_remote = False
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    # Enable cross compilation when connecting a remote device from a non-arm platform.
+    cross_compile = None
+    # cross_compile = "aarch64-linux-gnu-g++"
+
+    def __init__(self):
+        """Keep remote device for lifetime of object."""
+        self.device = self._get_remote()
+
+    @classmethod
+    def _get_remote(cls):
+        """Get a remote (or local) device to use for testing."""
+        if cls.use_remote:
+            # Here you may adjust settings to run the ACL unit tests via a remote
+            # device using the RPC mechanism. Use this in the case you want to compile
+            # an ACL module on a different machine to what you run the module on i.e.
+            # x86 -> AArch64.
+            #
+            # Use the following to connect directly to a remote device:
+            # device = rpc.connect(
+            #     hostname="0.0.0.0",
+            #     port=9090)
+            #
+            # Or connect via a tracker:
+            # device = tvm.autotvm.measure.request_remote(
+            #     host="0.0.0.0",
+            #     port=9090,
+            #     device_key="device_key",
+            #     timeout=1000)
+            #
+            # return device
+            raise NotImplementedError(
+                "Please adjust these settings to connect to your remote device.")
+        else:
+            device = rpc.LocalSession()
+            return device
+
+
+def skip_runtime_test():
+    """Skip test if it requires the runtime and it's not present."""
+    # ACL codegen not present.
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+    # Remote device is in use or ACL runtime not present
+    if not Device.use_remote and not arm_compute_lib.is_arm_compute_runtime_enabled():
+        print("Skip because runtime isn't present or a remote device isn't being used.")
+        return True
+
+
+def skip_codegen_test():
+    """Skip test if it requires the ACL codegen and it's not present."""
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+
+def build_module(mod, target, params=None, enable_acl=True):
+    """Build module with option to build for ACL."""
+    if isinstance(mod, tvm.relay.expr.Call):
+        mod = tvm.IRModule.from_expr(mod)
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        if enable_acl:
+            mod = arm_compute_lib.partition_for_arm_compute_lib(mod, params)
+        relay.backend.compile_engine.get().clear()
+        return relay.build(mod, target=target, params=params)
+
+
+def build_and_run(mod, inputs, outputs, params, device, enable_acl=True, no_runs=1):
+    """Build and run the relay module."""
+    graph, lib, params = build_module(mod, device.target, params, enable_acl)
+    lib = update_lib(lib, device.device, device.cross_compile)
+    gen_module = graph_runtime.create(graph, lib, ctx=device.device.cpu(0))
+    gen_module.set_input(**inputs)
+    gen_module.set_input(**params)

Review comment:
       Could remove ```gen_module.set_input(**params)``` in new behavior 

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        json, lib, params = relay.build(module, target=target)
+
+
+Export the module.
+
+.. code:: python
+
+    lib_path = '~/lib_acl.so'
+    cross_compile = 'aarch64-linux-gnu-c++'
+    lib.export_library(lib_path, cc=cross_compile)
+
+
+Run Inference. This must be on an Arm device. If compiling on x86 device and running on aarch64
+consider using the RPC mechanism.
+
+.. code:: python
+
+    tvm.runtime.load_module('lib_acl.so')
+    gen_module = tvm.contrib.graph_runtime.create(json, lib, ctx)

Review comment:
       ```python
   loaded_lib = tvm.runtime.load_module('lib_acl.so')
   gen_module = tvm.contrib.graph_runtime.GraphModule(loaded_lib['default'](ctx))
   ```
   
   P.S. Where is `ctx` define?

##########
File path: tests/python/contrib/test_arm_compute_lib/infrastructure.py
##########
@@ -0,0 +1,167 @@
+# 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.
+from itertools import zip_longest, combinations
+import json
+
+import tvm
+from tvm import relay
+from tvm import rpc
+from tvm.contrib import graph_runtime
+from tvm.relay.op.contrib import arm_compute_lib
+from tvm.contrib import util
+
+
+class Device:
+    """Adjust the following settings to connect to and use a remote device for tests."""
+    use_remote = False
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    # Enable cross compilation when connecting a remote device from a non-arm platform.
+    cross_compile = None
+    # cross_compile = "aarch64-linux-gnu-g++"
+
+    def __init__(self):
+        """Keep remote device for lifetime of object."""
+        self.device = self._get_remote()
+
+    @classmethod
+    def _get_remote(cls):
+        """Get a remote (or local) device to use for testing."""
+        if cls.use_remote:
+            # Here you may adjust settings to run the ACL unit tests via a remote
+            # device using the RPC mechanism. Use this in the case you want to compile
+            # an ACL module on a different machine to what you run the module on i.e.
+            # x86 -> AArch64.
+            #
+            # Use the following to connect directly to a remote device:
+            # device = rpc.connect(
+            #     hostname="0.0.0.0",
+            #     port=9090)
+            #
+            # Or connect via a tracker:
+            # device = tvm.autotvm.measure.request_remote(
+            #     host="0.0.0.0",
+            #     port=9090,
+            #     device_key="device_key",
+            #     timeout=1000)
+            #
+            # return device
+            raise NotImplementedError(
+                "Please adjust these settings to connect to your remote device.")
+        else:
+            device = rpc.LocalSession()
+            return device
+
+
+def skip_runtime_test():
+    """Skip test if it requires the runtime and it's not present."""
+    # ACL codegen not present.
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+    # Remote device is in use or ACL runtime not present
+    if not Device.use_remote and not arm_compute_lib.is_arm_compute_runtime_enabled():
+        print("Skip because runtime isn't present or a remote device isn't being used.")
+        return True
+
+
+def skip_codegen_test():
+    """Skip test if it requires the ACL codegen and it's not present."""
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+
+def build_module(mod, target, params=None, enable_acl=True):
+    """Build module with option to build for ACL."""
+    if isinstance(mod, tvm.relay.expr.Call):
+        mod = tvm.IRModule.from_expr(mod)
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        if enable_acl:
+            mod = arm_compute_lib.partition_for_arm_compute_lib(mod, params)
+        relay.backend.compile_engine.get().clear()
+        return relay.build(mod, target=target, params=params)
+
+
+def build_and_run(mod, inputs, outputs, params, device, enable_acl=True, no_runs=1):
+    """Build and run the relay module."""
+    graph, lib, params = build_module(mod, device.target, params, enable_acl)

Review comment:
       ```python
   lib = build_module(mod, device.target, params, enable_acl)
   ```

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,77 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,
+                                                                      size_t alignment) {
+  return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(size, alignment);
+}
+
+ACLMemoryRegion::ACLMemoryRegion(size_t size, size_t alignment)
+    : IMemoryRegion(size), ptr_(nullptr) {
+  if (size != 0) {
+    this->ptr_ = this->device_api_->AllocDataSpace(this->ctx_, size, alignment, {});
+  }
+}
+
+ACLMemoryRegion::ACLMemoryRegion(void* ptr, size_t size)
+    : IMemoryRegion(size), ptr_(nullptr), is_subregion_(true) {
+  if (size != 0) {
+    this->ptr_ = ptr;
+  }
+}
+
+ACLMemoryRegion::~ACLMemoryRegion() {
+  if (this->ptr_ != nullptr && !is_subregion_) {
+    this->device_api_->FreeDataSpace(this->ctx_, this->ptr_);
+  }
+}
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLMemoryRegion::extract_subregion(size_t offset,
+                                                                               size_t size) {
+  if (this->ptr_ != nullptr && (offset < _size) && (_size - offset >= size)) {
+    return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(

Review comment:
       TVM has used C++14. We could use `std::make_unique` here.

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        json, lib, params = relay.build(module, target=target)
+
+
+Export the module.
+
+.. code:: python
+
+    lib_path = '~/lib_acl.so'
+    cross_compile = 'aarch64-linux-gnu-c++'
+    lib.export_library(lib_path, cc=cross_compile)
+
+
+Run Inference. This must be on an Arm device. If compiling on x86 device and running on aarch64
+consider using the RPC mechanism.
+
+.. code:: python
+
+    tvm.runtime.load_module('lib_acl.so')
+    gen_module = tvm.contrib.graph_runtime.create(json, lib, ctx)
+    d_data = np.random.uniform(0, 1, data_shape).astype(data_type)
+    map_inputs = {'data': d_data}
+    gen_module.map_inputs(**map_inputs)

Review comment:
       Where is definition of `map_inputs`? `set_input`? 




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        json, lib, params = relay.build(module, target=target)
+
+
+Export the module.
+
+.. code:: python
+
+    lib_path = '~/lib_acl.so'
+    cross_compile = 'aarch64-linux-gnu-c++'
+    lib.export_library(lib_path, cc=cross_compile)
+
+
+Run Inference. This must be on an Arm device. If compiling on x86 device and running on aarch64
+consider using the RPC mechanism.
+
+.. code:: python
+
+    tvm.runtime.load_module('lib_acl.so')
+    gen_module = tvm.contrib.graph_runtime.create(json, lib, ctx)

Review comment:
       Added `ctx = tvm.cpu(0)`




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   


----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: cmake/modules/contrib/ACL.cmake
##########
@@ -0,0 +1,68 @@
+# 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.
+
+# We separate the codegen and runtime build since ACL can only be built
+# for AArch. In the world where we take the cross compilation approach,
+# which is common with arm devices, we need to be able to cross-compile
+# a relay graph on x86 for AArch and then run the graph on AArch.

Review comment:
       I am not familiar with runtime compilation. What I know is that for edge devices we can run `make runtime` to compile only the runtime required modules. Could we do the same for ACL as well?

##########
File path: src/relay/backend/contrib/acl/README.md
##########
@@ -0,0 +1,111 @@
+<!---
+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 Arm&reg; Compute Library Integration

Review comment:
       It would be better to put this in the document (and use RST doc style) so that people can easily find it.

##########
File path: src/relay/backend/contrib/acl/README.md
##########
@@ -0,0 +1,111 @@
+<!---
+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 Arm&reg; Compute Library Integration
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+## Building with ACL support
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ACL=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_GRAPH_RUNTIME_ACL=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+## Usage
+_Note:_ this may not stay up-to-date with changes to the API.
+1. Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+    ```
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides,
+                              layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+    ```
+2. Annotate and partition the graph for ACL.

Review comment:
       Are we expecting users to call a sequence of passes by themselves, especially you already have `partition_for_acl`?




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   Thanks @lhutton1 and everyone. 


----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64

Review comment:
       - s/USE_ACL/USE_ARM_COMPUTE_LIB/
   - s/USE_GRAPH_RUNTIME_ACL/USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME/

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.

Review comment:
       Use the RST note format:
   
   ```rst
   .. note::
   
       This section...
   ```

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        json, lib, params = relay.build(module, target=target)
+
+
+Export the module.
+
+.. code:: python
+
+    lib_path = '~/lib_acl.so'
+    cross_compile = 'aarch64-linux-gnu-c++'
+    lib.export_library(lib_path, cc=cross_compile)
+
+
+Run Inference. This must be on an Arm device. If compiling on x86 device and running on aarch64
+consider using the RPC mechanism.

Review comment:
       s/consider/, consider/

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB

Review comment:
       I think what zhiics meeant was that we can reduce the number of macro blocks.
   - If we move CachedLayout struct definition in the ACLRuntime as its inner struct, then we can merge the first block (L48) to the third block (L240).
   - We can also change the third block (L240) to the following form so that we can remove the second block (L185)
   ```c
   #ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
   void BuildEngine() {
     // Build engine.
   }
   #else
   void BuildEngine() {
     // Do nothing.
   }
   #endif
   ```
   
   As a result, we only need 2 macro blocks in this file.

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single

Review comment:
       s/whilst/whitelist/
   
   p.s. maybe we should avoid using "whitelist" in the future orz




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: cmake/modules/contrib/ArmComputeLib.cmake
##########
@@ -0,0 +1,66 @@
+# 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.
+
+# We separate the codegen and runtime build since ACL can only be built
+# for AArch. In the world where we take the cross compilation approach,
+# which is common with arm devices, we need to be able to cross-compile
+# a relay graph on x86 for AArch and then run the graph on AArch.
+if(USE_ARM_COMPUTE_LIB)
+    file(GLOB ACL_RELAY_CONTRIB_SRC src/relay/backend/contrib/arm_compute_lib/*.cc)
+    file(GLOB ACL_RUNTIME_MODULE src/runtime/contrib/arm_compute_lib/acl_runtime.cc)
+    list(APPEND COMPILER_SRCS ${ACL_RELAY_CONTRIB_SRC})
+    list(APPEND COMPILER_SRCS ${ACL_RUNTIME_MODULE})
+    message(STATUS "Build with Arm Compute Library support...")
+endif()
+
+if(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME)
+    set(ACL_PATH ${CMAKE_CURRENT_SOURCE_DIR}/acl)
+    # Detect custom ACL path.
+    if (NOT USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME STREQUAL "ON")
+        set(ACL_PATH ${USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME})
+    endif()
+
+    file(GLOB ACL_CONTRIB_SRC src/runtime/contrib/arm_compute_lib/*)
+
+    set(ACL_INCLUDE_DIRS ${ACL_PATH}/include ${ACL_PATH})
+    include_directories(${ACL_INCLUDE_DIRS})
+
+    find_library(EXTERN_ACL_COMPUTE_LIB
+            NAMES arm_compute libarm_compute
+            HINTS "${ACL_PATH}" "${ACL_PATH}/lib" "${ACL_PATH}/build"
+            )
+    find_library(EXTERN_ACL_COMPUTE_CORE_LIB
+            NAMES arm_compute_core libarm_compute_core
+            HINTS "${ACL_PATH}" "${ACL_PATH}/lib" "${ACL_PATH}/build"
+            )
+    find_library(EXTERN_ACL_COMPUTE_GRAPH_LIB
+            NAMES arm_compute_graph libarm_compute_graph
+            HINTS "${ACL_PATH}" "${ACL_PATH}/lib" "${ACL_PATH}/build"
+            )
+
+    list(APPEND TVM_RUNTIME_LINKER_LIBS ${EXTERN_ACL_COMPUTE_LIB})
+    list(APPEND TVM_RUNTIME_LINKER_LIBS ${EXTERN_ACL_COMPUTE_CORE_LIB})
+    list(APPEND TVM_RUNTIME_LINKER_LIBS ${EXTERN_ACL_COMPUTE_GRAPH_LIB})
+    list(APPEND RUNTIME_SRCS ${ACL_CONTRIB_SRC})
+    message(STATUS "Build with Arm Compute Library graph runtime support: "
+            ${EXTERN_ACL_COMPUTE_LIB} ", \n"
+            ${EXTERN_ACL_COMPUTE_CORE_LIB} ", \n"
+            ${EXTERN_ACL_COMPUTE_GRAPH_LIB})
+
+    # Set flag to detect ACL graph runtime support.
+    add_definitions(-DTVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB)

Review comment:
       See comment below




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB

Review comment:
       I think we need these as acl_runtime.cc must be able to be compiled on an x86 machine so that codegen can construct an ACL runtime module. Without these guards, we would attempt to compile code from ACL which can't be done on an x86 machine.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB

Review comment:
       Ah I see, thanks, I'll change 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 a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/acl/README.md
##########
@@ -0,0 +1,111 @@
+<!---
+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 Arm&reg; Compute Library Integration

Review comment:
       Good question. I'm looking at "tutorials" or "deploy". @tqchen do you have a preference?




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      uint32_t eid = EntryID(nid, 0);
+      if (nodes_[nid].GetOpType() == "input") {
+        void* data = data_entry_[eid]->data;
+        CheckACLError(layer_.inputs[i].allocator()->import_memory(data));
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      void* data = data_entry_[eid]->data;
+      CheckACLError(layer_.outputs[i].allocator()->import_memory(data));
+    }
+
+    this->layer_.function->run();
+#else
+    LOG(FATAL) << "Cannot call run on Arm Compute Library module without runtime enabled. "
+               << "Please build with USE_ACL_GRAPH_RUNTIME.";
+#endif
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {
+    if (format == "json") {
+      return graph_json_;
+    }
+    LOG(FATAL) << "Format not supported by Arm Compute Library runtime.";
+    return "";
+  }
+
+ private:
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+  /*!
+   * \brief Build ACL layer from JSON representation and cache.
+   *
+   * \note For the time being only one layer or operator is supported
+   * per engine.
+   */
+  void BuildEngine() {
+    std::shared_ptr<arm_compute::MemoryManagerOnDemand> mm = MakeMemoryManager();
+    int num_pools = 0;
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      uint32_t nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "input") {
+        layer_.inputs.push_back(MakeTensor(node));
+      } else if (node.GetOpType() == "const") {
+        uint32_t eid = EntryID(nid, 0);
+        void* data = data_entry_[eid]->data;
+        layer_.const_inputs.push_back(MakeTensor(node, data));
+      }
+    }
+
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "kernel") {
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name || "arm_compute_lib.conv2d" == op_name) {
+          CreateConvolution2DLayer(&layer_, node, mm);
+          num_pools++;
+        } else if ("nn.max_pool2d" == op_name) {
+          CreatePoolingLayer(&layer_, node);
+        } else if ("reshape" == op_name) {
+          CreateReshapeLayer(&layer_, node);
+        } else {
+          LOG(FATAL) << "Unsupported op: " << op_name;
+        }
+        // Only expect one op for the time being
+        break;

Review comment:
       Good catch, thanks




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   Please note this PR depends on #5919.


----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,143 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const ConstantNode* cn) override;
+
+  /*!
+   * \brief Get the constant data transposed when pre-processing the
+   * input function.
+   *
+   * \return An array of constants
+   */
+  Array<runtime::NDArray> GetParamsData();
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOp(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvolution(const CallNode* cn);
+
+  /* \brief Transposed constant tensors to serialize. Arm Compute Library expects constant tensors
+   * in OHWI format. */
+  Array<runtime::NDArray> constants_;
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only currently supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);
+
+/*!
+ * \brief Get the external symbol of the Relay function name.
+ *
+ * \param func The provided function.
+ *
+ * \return An external symbol.
+ */
+std::string GetExtSymbol(const Function& func) {

Review comment:
       Sounds good, is there any preference on this being 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] mbaret commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,122 @@
+# 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
+"""Arm Compute Library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_enabled():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.MergeComposite(arm_compute_lib_pattern_table()),
+                                    transform.AnnotateTarget('arm_compute_lib'),
+                                    transform.PartitionGraph()])
+
+    return seq(mod)
+
+
+@register_pattern_table("arm_compute_lib")
+def arm_compute_lib_pattern_table():
+    """Get the ACL pattern table."""
+
+    def conv_pattern():
+        """Create a convolution pattern.
+
+        Returns
+        -------
+        pattern : dataflow_pattern.AltPattern
+            Denotes the convolution pattern.
+        """
+        pattern = is_op('nn.pad')(wildcard()) | wildcard()
+        pattern = is_op('nn.conv2d')(pattern, is_constant())
+        pattern = pattern.optional(lambda x: is_op('nn.bias_add')(x, is_constant()))
+        pattern = pattern.optional(is_op('nn.relu'))
+        return pattern
+
+    def check_conv(extract):
+        """Check conv pattern is supported by ACL."""
+        call = extract
+        while call.op.name != "nn.conv2d":
+            call = call.args[0]
+        return conv2d(call.attrs, call.args)
+
+    return [('arm_compute_lib.conv2d', conv_pattern(), check_conv)]
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.arm_compute_lib")
+    def _func_wrapper(attrs, args):
+        return supported
+
+    return _func_wrapper
+
+
+_register_external_op_helper("reshape")
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.arm_compute_lib")
+def conv2d(attrs, args):
+    """Check if the external ACL codegen for conv2d should be used."""
+    if attrs.groups != 1:
+        return False
+    if attrs.data_layout != "NHWC":
+        return False

Review comment:
       Pedantically you may want to check for data type here as well (presumably = float32).

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc

Review comment:
       codegen_acl.cc -> codegen.cc

##########
File path: src/runtime/contrib/arm_compute_lib/acl_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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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 ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL 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.";
+    SetupConstants(consts);
+    BuildEngine();
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {
+    if (format == "json") {
+      return graph_json_;
+    }
+    LOG(FATAL) << "Format not supported by Arm Compute Library runtime.";
+    return "";
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      uint32_t eid = EntryID(nid, 0);
+      if (nodes_[nid].GetOpType() == "input") {
+        void* data = data_entry_[eid]->data;
+        CheckACLError(layer_.inputs[i].allocator()->import_memory(data));
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      void* data = data_entry_[eid]->data;
+      CheckACLError(layer_.outputs[i].allocator()->import_memory(data));
+    }
+
+    this->layer_.function->run();
+  }
+
+ private:
+  /*!
+   * \brief Build ACL layer from JSON representation and cache.
+   *
+   * \note For the time being only one layer or operator is supported
+   * per engine.
+   */
+  void BuildEngine() {
+    std::shared_ptr<arm_compute::MemoryManagerOnDemand> mm = MakeMemoryManager();
+    int num_pools = 0;
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      uint32_t nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "input") {
+        layer_.inputs.push_back(MakeTensor(node));
+      } else if (node.GetOpType() == "const") {
+        uint32_t eid = EntryID(nid, 0);
+        void* data = data_entry_[eid]->data;
+        layer_.const_inputs.push_back(MakeTensor(node, data));
+      }
+    }
+
+    bool found_kernel_node = false;
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (found_kernel_node) {
+        LOG(FATAL)
+            << "Arm Compute Library runtime module only supports one kernel node per function.";
+      }
+      if (node.GetOpType() == "kernel") {
+        found_kernel_node = true;
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name || "arm_compute_lib.conv2d" == op_name) {

Review comment:
       Why either or here? Could all conv2ds not be arm_compute_lib.conv2ds?

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOpJSONNode(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    name = comp.value();
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvJSONNode(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+  } else {
+    LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+               << cn->op->GetTypeKey();
+  }
+
+  return AddNode(json_node, GetRef<Expr>(cn));
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateOpJSONNode(const CallNode* cn) {
+  const auto* op = cn->op.as<OpNode>();
+  CHECK(op);
+  const std::string name = op->name;
+  // Collect inputs
+  std::vector<JSONGraphNodeEntry> inputs;
+  for (const auto& arg : cn->args) {
+    auto res = VisitExpr(arg);
+    inputs.insert(inputs.end(), res.begin(), res.end());
+  }
+  // Create JSON op
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, cn);
+  return json_node;
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateCompositeConvJSONNode(const CallNode* cn) {
+  const std::string name = "arm_compute_lib.conv2d";
+  const CallNode* pad = nullptr;
+  const CallNode* conv;
+  const CallNode* bias = nullptr;
+  bool has_activation = false;
+
+  // Unpack composite function
+  const auto* fn = cn->op.as<FunctionNode>();
+  CHECK(fn);
+  const auto* current_call = fn->body.as<CallNode>();
+  if (backend::IsOp(current_call, "nn.relu")) {
+    has_activation = true;
+    current_call = current_call->args[0].as<CallNode>();
+  }
+  if (backend::IsOp(current_call, "nn.bias_add")) {
+    bias = current_call;
+    current_call = current_call->args[0].as<CallNode>();
+  }
+  CHECK(backend::IsOp(current_call, "nn.conv2d"));
+  conv = current_call;
+  if (!current_call->args.empty() && current_call->args[0]->IsInstance<CallNode>()) {
+    current_call = current_call->args[0].as<CallNode>();
+    if (backend::IsOp(current_call, "nn.pad")) {
+      pad = current_call;
+    }
+  }
+
+  const auto* conv_attr = conv->attrs.as<Conv2DAttrs>();
+  CHECK(conv_attr);
+  CHECK(conv_attr->kernel_layout == "OHWI")
+      << "Kernel layout must be OHWI, has the module been pre-processed correctly?";
+
+  std::vector<JSONGraphNodeEntry> inputs;
+  inputs.push_back(VisitExpr(cn->args[0])[0]);
+  inputs.push_back(VisitExpr(conv->args[1])[0]);
+  if (bias) {
+    inputs.push_back(VisitExpr(bias->args[1])[0]);
+  }
+
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, conv);
+
+  // Override attributes
+  if (pad) {
+    const auto* pad_attr = pad->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    // Convert to TVM layout for now, conversion to ACL layout takes place in runtime.
+    // Standard convolution pad layout for TVM: top, left, bottom, right.
+    std::vector<std::string> padding = {std::to_string(p[1][0].as<IntImmNode>()->value),
+                                        std::to_string(p[2][0].as<IntImmNode>()->value),
+                                        std::to_string(p[1][1].as<IntImmNode>()->value),
+                                        std::to_string(p[2][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    json_node->SetAttr("padding", padding_attr);
+  }
+  if (has_activation) {
+    std::vector<std::string> activation_type = {"relu"};
+    std::vector<dmlc::any> act_attr;
+    act_attr.emplace_back(activation_type);
+    json_node->SetAttr("activation_type", act_attr);
+  }
+  return json_node;
+}
+
+IRModule PreProcessModule(const IRModule& mod) {
+  IRModule preprocessed_module;
+  tvm::Map<String, Array<String>> desired_layouts = {{"nn.conv2d", {"NHWC", "OHWI"}}};
+  preprocessed_module = transform::ConvertLayout(desired_layouts)(mod);
+  preprocessed_module = transform::FoldConstant()(preprocessed_module);
+  return preprocessed_module;
+}

Review comment:
       Does this mean if ACL offload happens the entire graph (including bits not destined for ACL) get converted to NHWC, or is it just ACL partitions?

##########
File path: src/runtime/contrib/arm_compute_lib/acl_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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif

Review comment:
       Not sure I understand 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] lhutton1 commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOpJSONNode(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";

Review comment:
       Currently functions are not supported by JSON codegen, although we can always change this functionality in the ACL codegen. I think it's ok for now to support only composite functions but we could add this in the future.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,220 @@
+/*
+ * 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/arm_compute_lib/codegen.cc
+ * \brief Implementation of the Relay -> ACL 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"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  /*!
+   * \brief Visit call nodes and generate appropriate JSON node.
+   *
+   * \param cn The current call node.
+   * \return A list of graph entry nodes.
+   */
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override {
+    if (cn->op.as<OpNode>()) {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+    if (!cn->op.as<FunctionNode>()) {
+      LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+                 << cn->op->GetTypeKey();
+    }
+    auto fn = cn->op.as<FunctionNode>();
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    const std::string name = comp.value();
+    std::shared_ptr<JSONGraphNode> json_node;
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvJSONNode(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+    return AddNode(json_node, GetRef<Expr>(cn));
+  }
+
+ private:
+  /*!
+   * \brief Create a JSON representation of a composite convolution.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvJSONNode(const CallNode* cn) {
+    const std::string name = "nn.conv2d";
+    const CallNode* pad = nullptr;
+    const CallNode* conv;

Review comment:
       My intention here was to express that conv should always be initialized, whilst pad and bias are optional. Happy to change it 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] lhutton1 commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single

Review comment:
       Should be while, seems like whilst is more of a British term :) 




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: cmake/modules/contrib/ACL.cmake
##########
@@ -0,0 +1,68 @@
+# 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.
+
+# We separate the codegen and runtime build since ACL can only be built
+# for AArch. In the world where we take the cross compilation approach,
+# which is common with arm devices, we need to be able to cross-compile
+# a relay graph on x86 for AArch and then run the graph on AArch.

Review comment:
       I see. Make sense.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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 ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL 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.";
+    SetupConstants(consts);
+    BuildEngine();
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {
+    if (format == "json") {
+      return graph_json_;
+    }
+    LOG(FATAL) << "Format not supported by Arm Compute Library runtime.";
+    return "";
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      uint32_t eid = EntryID(nid, 0);
+      if (nodes_[nid].GetOpType() == "input") {
+        void* data = data_entry_[eid]->data;
+        CheckACLError(layer_.inputs[i].allocator()->import_memory(data));
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      void* data = data_entry_[eid]->data;
+      CheckACLError(layer_.outputs[i].allocator()->import_memory(data));
+    }
+
+    this->layer_.function->run();
+  }
+
+ private:
+  /*!
+   * \brief Build ACL layer from JSON representation and cache.
+   *
+   * \note For the time being only one layer or operator is supported
+   * per engine.
+   */
+  void BuildEngine() {
+    std::shared_ptr<arm_compute::MemoryManagerOnDemand> mm = MakeMemoryManager();
+    int num_pools = 0;
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      uint32_t nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "input") {
+        layer_.inputs.push_back(MakeTensor(node));
+      } else if (node.GetOpType() == "const") {
+        uint32_t eid = EntryID(nid, 0);
+        void* data = data_entry_[eid]->data;
+        layer_.const_inputs.push_back(MakeTensor(node, data));
+      }
+    }
+
+    bool found_kernel_node = false;
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (found_kernel_node) {
+        LOG(FATAL)
+            << "Arm Compute Library runtime module only supports one kernel node per function.";
+      }
+      if (node.GetOpType() == "kernel") {
+        found_kernel_node = true;
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name || "arm_compute_lib.conv2d" == op_name) {

Review comment:
       Yes they could all be `nn.conv2d`, when writing I thought there may be a case where we want to distinguish between a composite operator and a single operator, although I don't think this will happen.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single

Review comment:
       Ah that makes much more sense...




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: cmake/config.cmake
##########
@@ -184,6 +184,18 @@ set(USE_SORT ON)
 # Whether use MKL-DNN (DNNL) codegen
 set(USE_DNNL_CODEGEN OFF)
 
+# Whether to use Arm Compute Library (ACL) codegen
+# We provide 2 separate flags since we cannot build the ACL runtime on x86.
+# This is useful for cases where you want to cross-compile a relay graph
+# on x86 then run on AArch.

Review comment:
       I added the location of the tutorial, I think that will be sufficient




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOpJSONNode(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    name = comp.value();
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvJSONNode(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+  } else {
+    LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+               << cn->op->GetTypeKey();
+  }
+
+  return AddNode(json_node, GetRef<Expr>(cn));
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateOpJSONNode(const CallNode* cn) {
+  const auto* op = cn->op.as<OpNode>();
+  CHECK(op);
+  const std::string name = op->name;
+  // Collect inputs
+  std::vector<JSONGraphNodeEntry> inputs;
+  for (const auto& arg : cn->args) {
+    auto res = VisitExpr(arg);
+    inputs.insert(inputs.end(), res.begin(), res.end());
+  }
+  // Create JSON op
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, cn);
+  return json_node;
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateCompositeConvJSONNode(const CallNode* cn) {
+  const std::string name = "arm_compute_lib.conv2d";
+  const CallNode* pad = nullptr;
+  const CallNode* conv;
+  const CallNode* bias = nullptr;
+  bool has_activation = false;
+
+  // Unpack composite function
+  const auto* fn = cn->op.as<FunctionNode>();
+  CHECK(fn);
+  const auto* current_call = fn->body.as<CallNode>();
+  if (backend::IsOp(current_call, "nn.relu")) {
+    has_activation = true;
+    current_call = current_call->args[0].as<CallNode>();
+  }
+  if (backend::IsOp(current_call, "nn.bias_add")) {
+    bias = current_call;
+    current_call = current_call->args[0].as<CallNode>();
+  }
+  CHECK(backend::IsOp(current_call, "nn.conv2d"));
+  conv = current_call;
+  if (!current_call->args.empty() && current_call->args[0]->IsInstance<CallNode>()) {
+    current_call = current_call->args[0].as<CallNode>();
+    if (backend::IsOp(current_call, "nn.pad")) {
+      pad = current_call;
+    }
+  }
+
+  const auto* conv_attr = conv->attrs.as<Conv2DAttrs>();
+  CHECK(conv_attr);
+  CHECK(conv_attr->kernel_layout == "OHWI")
+      << "Kernel layout must be OHWI, has the module been pre-processed correctly?";
+
+  std::vector<JSONGraphNodeEntry> inputs;
+  inputs.push_back(VisitExpr(cn->args[0])[0]);
+  inputs.push_back(VisitExpr(conv->args[1])[0]);
+  if (bias) {
+    inputs.push_back(VisitExpr(bias->args[1])[0]);
+  }
+
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, conv);
+
+  // Override attributes
+  if (pad) {
+    const auto* pad_attr = pad->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    // Convert to TVM layout for now, conversion to ACL layout takes place in runtime.
+    // Standard convolution pad layout for TVM: top, left, bottom, right.
+    std::vector<std::string> padding = {std::to_string(p[1][0].as<IntImmNode>()->value),
+                                        std::to_string(p[2][0].as<IntImmNode>()->value),
+                                        std::to_string(p[1][1].as<IntImmNode>()->value),
+                                        std::to_string(p[2][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    json_node->SetAttr("padding", padding_attr);
+  }
+  if (has_activation) {
+    std::vector<std::string> activation_type = {"relu"};
+    std::vector<dmlc::any> act_attr;
+    act_attr.emplace_back(activation_type);
+    json_node->SetAttr("activation_type", act_attr);
+  }
+  return json_node;
+}
+
+IRModule PreProcessModule(const IRModule& mod) {
+  IRModule preprocessed_module;
+  tvm::Map<String, Array<String>> desired_layouts = {{"nn.conv2d", {"NHWC", "OHWI"}}};
+  preprocessed_module = transform::ConvertLayout(desired_layouts)(mod);
+  preprocessed_module = transform::FoldConstant()(preprocessed_module);
+  return preprocessed_module;
+}

Review comment:
       Just the ACL partitions




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   @FrozenGene Can you take another and approve explicitly if it looks good 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] lhutton1 commented on pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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


   > let us consider expand the name acl to arm_compute_lib or some other alternatives, since ACL means different things to ML/NLP audiences
   
   I'd be happy to change this, `arm_compute_lib` sounds good to me, @u99127 do you have any other preference?


----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,74 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,
+                                                                      size_t alignment) {
+  return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(size, alignment);

Review comment:
       Thanks, not sure how I missed 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] lhutton1 commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,135 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+
+Usage
+-----
+
+*Note:* this section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        json, lib, params = relay.build(module, target=target)

Review comment:
       Thanks for the heads up on the api changes!




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,188 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOp(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    name = comp.value();
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvolution(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+  } else {
+    LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+               << cn->op->GetTypeKey();
+  }
+
+  return AddNode(json_node, GetRef<Expr>(cn));
+}
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const ConstantNode* cn) {
+  this->constants_.push_back(cn->data);
+  return JSONSerializer::VisitExpr_(cn);
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateOp(const CallNode* cn) {
+  const auto* op = cn->op.as<OpNode>();
+  CHECK(op);
+  const std::string name = op->name;
+  // Collect inputs
+  std::vector<JSONGraphNodeEntry> inputs;
+  for (const auto& arg : cn->args) {
+    auto res = VisitExpr(arg);
+    inputs.insert(inputs.end(), res.begin(), res.end());
+  }
+  // Create JSON op
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, cn);
+  return json_node;
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateCompositeConvolution(const CallNode* cn) {

Review comment:
       Ditto. s/CreateCompositeConvolution/CreateCompositeConvJSONNode/ ?

##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_present():

Review comment:
       `is_arm_compute_runtime_enabled` seems better to me.

##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_present():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    return tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : dict[str, NDArray]

Review comment:
       The right type for `params` should be `Optional[Dict[str, NDArray]]` (capital D for Dict) as it might be `None`.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_utils.cc
##########
@@ -0,0 +1,116 @@
+/*
+ * 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_utils.cc
+ * \brief Utils and common functions for the interface.
+ */
+
+#include "acl_utils.h"
+
+#include <arm_compute/runtime/OffsetLifetimeManager.h>
+#include <arm_compute/runtime/PoolManager.h>
+#include <tvm/runtime/data_type.h>
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+
+void CheckACLError(const arm_compute::Status& status) {
+  CHECK(status.error_code() == arm_compute::ErrorCode::OK) << "ACL: " << status.error_description();
+}
+
+arm_compute::Tensor MakeTensor(const JSONGraphNode& tensor_rep, void* data) {
+  CHECK(tensor_rep.GetOpType() == "input" || tensor_rep.GetOpType() == "const");
+  arm_compute::Tensor tensor;
+  arm_compute::TensorInfo info = MakeTensorInfo(tensor_rep.GetOpShape()[0]);
+  tensor.allocator()->init(info);
+  if (data != nullptr) {
+    CheckACLError(tensor.allocator()->import_memory(data));
+  }
+  return tensor;
+}
+
+arm_compute::Tensor MakeOutputTensor(const std::vector<int64_t>& shape) {
+  arm_compute::Tensor tensor;
+  tensor.allocator()->init(MakeTensorInfo(shape));
+  return tensor;
+}
+
+arm_compute::TensorInfo MakeTensorInfo(const std::vector<int64_t>& shape) {
+  arm_compute::TensorShape acl_shape = MakeTensorShape(shape);
+  return arm_compute::TensorInfo(acl_shape, 1, arm_compute::DataType::F32,
+                                 arm_compute::DataLayout::NHWC);
+}
+
+arm_compute::TensorShape MakeTensorShape(const std::vector<int64_t>& shape) {
+  arm_compute::TensorShape acl_shape;
+  for (unsigned int i = shape.size(); i > 0; --i) {
+    acl_shape.set(shape.size() - i, shape[i - 1]);
+  }
+  return acl_shape;
+}
+
+std::shared_ptr<arm_compute::MemoryManagerOnDemand> MakeMemoryManager() {
+  auto lifetime_mgr = std::make_shared<arm_compute::OffsetLifetimeManager>();
+  auto pool_mgr = std::make_shared<arm_compute::PoolManager>();
+  return std::make_shared<arm_compute::MemoryManagerOnDemand>(lifetime_mgr, pool_mgr);
+}
+
+arm_compute::PadStrideInfo ToACLPadStride(const std::vector<std::string>& pad,
+                                          const std::vector<std::string>& stride) {
+  int pad_0, pad_1, pad_2, pad_3;
+
+  size_t size = pad.size();
+  if (size == 1) {
+    int pad_v = std::stoi(pad[0]);
+    pad_0 = pad_v;
+    pad_1 = pad_v;
+    pad_2 = pad_v;
+    pad_3 = pad_v;
+  } else if (size == 2) {
+    // TVM: height, width -> ACL: left, right, top, bottom
+    int pad_h = std::stoi(pad[0]);
+    int pad_w = std::stoi(pad[1]);
+    pad_0 = pad_w;
+    pad_1 = pad_w;
+    pad_2 = pad_h;
+    pad_3 = pad_h;
+  } else if (size == 4) {
+    // TVM: top, left, bottom, right -> ACL: left, right, top, bottom
+    pad_0 = std::stoi(pad[1]);
+    pad_1 = std::stoi(pad[3]);
+    pad_2 = std::stoi(pad[0]);
+    pad_3 = std::stoi(pad[2]);
+  } else {
+    LOG(FATAL) << "Unsupported padding dimensions";
+    return arm_compute::PadStrideInfo();

Review comment:
       FATAL will throw exception and crash the execution so you don't need this return.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      uint32_t eid = EntryID(nid, 0);
+      if (nodes_[nid].GetOpType() == "input") {
+        void* data = data_entry_[eid]->data;
+        CheckACLError(layer_.inputs[i].allocator()->import_memory(data));
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      void* data = data_entry_[eid]->data;
+      CheckACLError(layer_.outputs[i].allocator()->import_memory(data));
+    }
+
+    this->layer_.function->run();
+#else
+    LOG(FATAL) << "Cannot call run on Arm Compute Library module without runtime enabled. "
+               << "Please build with USE_ACL_GRAPH_RUNTIME.";
+#endif
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {
+    if (format == "json") {
+      return graph_json_;
+    }
+    LOG(FATAL) << "Format not supported by Arm Compute Library runtime.";
+    return "";
+  }
+
+ private:
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+  /*!
+   * \brief Build ACL layer from JSON representation and cache.
+   *
+   * \note For the time being only one layer or operator is supported
+   * per engine.
+   */
+  void BuildEngine() {
+    std::shared_ptr<arm_compute::MemoryManagerOnDemand> mm = MakeMemoryManager();
+    int num_pools = 0;
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      uint32_t nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "input") {
+        layer_.inputs.push_back(MakeTensor(node));
+      } else if (node.GetOpType() == "const") {
+        uint32_t eid = EntryID(nid, 0);
+        void* data = data_entry_[eid]->data;
+        layer_.const_inputs.push_back(MakeTensor(node, data));
+      }
+    }
+
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "kernel") {
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name || "arm_compute_lib.conv2d" == op_name) {
+          CreateConvolution2DLayer(&layer_, node, mm);
+          num_pools++;
+        } else if ("nn.max_pool2d" == op_name) {
+          CreatePoolingLayer(&layer_, node);
+        } else if ("reshape" == op_name) {
+          CreateReshapeLayer(&layer_, node);
+        } else {
+          LOG(FATAL) << "Unsupported op: " << op_name;
+        }
+        // Only expect one op for the time being
+        break;

Review comment:
       Didn't catch this comment. You meant the current ACL runtime only supports one kernel node? If so you should error out if there are more than one kernel nodes; otherwise the functionality is incorrect.

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,143 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const ConstantNode* cn) override;
+
+  /*!
+   * \brief Get the constant data transposed when pre-processing the
+   * input function.
+   *
+   * \return An array of constants
+   */
+  Array<runtime::NDArray> GetParamsData();
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOp(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvolution(const CallNode* cn);
+
+  /* \brief Transposed constant tensors to serialize. Arm Compute Library expects constant tensors
+   * in OHWI format. */
+  Array<runtime::NDArray> constants_;
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only currently supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);
+
+/*!
+ * \brief Get the external symbol of the Relay function name.
+ *
+ * \param func The provided function.
+ *
+ * \return An external symbol.
+ */
+std::string GetExtSymbol(const Function& func) {

Review comment:
       This is implemented in `CSourceModuleCodegenBase` already. Maybe we can move this function out of `CSourceModuleCodegenBase` so that it can be used by all backend under contrib.
   
   cc @zhiics 

##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_present():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    return tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : dict[str, NDArray]
+        Constant input parameters.
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.MergeComposite(pattern_table()),
+                                    transform.AnnotateTarget('arm_compute_lib'),
+                                    transform.PartitionGraph()])
+
+    return seq(mod)
+
+
+@register_pattern_table("arm_compute_lib")
+def pattern_table():

Review comment:
       It's better to improve the naming of this function as it is called directly in this file in addition to register to the global pattern table. Something like `arm_compute_lib_pattern_table` for `get_arm_compute_lib_pattern_table` would be better to me.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }

Review comment:
       it seems to me that even the constants are transposed by the preprocess sequence, you can still leave them to MetadataModule? If that's the case, you can reuse `Init(const Array<NDArray>& constants)`, and you don't have to save/load constants from binary.

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,188 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOp(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    name = comp.value();
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvolution(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+  } else {
+    LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+               << cn->op->GetTypeKey();
+  }
+
+  return AddNode(json_node, GetRef<Expr>(cn));
+}
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const ConstantNode* cn) {
+  this->constants_.push_back(cn->data);
+  return JSONSerializer::VisitExpr_(cn);
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateOp(const CallNode* cn) {

Review comment:
       Would s/CreateOp/CreateOpJSONNode/ be better?




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB

Review comment:
       I think we need these as acl_runtime.cc must be able to be compiled on an x86 machine so that codegen can construct an ACL runtime module. Without these guards, we would attempt to compile code from ACL which can't be done.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,76 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,

Review comment:
       Since we subclass ACL's Allocator interface here, these function names cannot be changed




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   > Not too sure what happened to the Mac build, looks unrelated.
   
   Should not be an issue. You could use `git commit --allow-empty` to re-trigger 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] lhutton1 commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: cmake/modules/contrib/ACL.cmake
##########
@@ -0,0 +1,68 @@
+# 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.
+
+# We separate the codegen and runtime build since ACL can only be built
+# for AArch. In the world where we take the cross compilation approach,
+# which is common with arm devices, we need to be able to cross-compile
+# a relay graph on x86 for AArch and then run the graph on AArch.

Review comment:
       Yes we do that currently, you just need to add USE_ACL_GRAPH_RUNTIME to the cmake config. The reason for the separation is based around the fact that we want to give the user the option to only compile the codegen part. This way they can compile an ACL module without needing to have ACL present on an x86 device

##########
File path: src/relay/backend/contrib/acl/README.md
##########
@@ -0,0 +1,111 @@
+<!---
+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 Arm&reg; Compute Library Integration

Review comment:
       Makes sense, would somewhere under docs be the best place to put this? Perhaps `docs/backend/contrib` or `docs/runtime/contrib`?

##########
File path: src/relay/backend/contrib/acl/README.md
##########
@@ -0,0 +1,111 @@
+<!---
+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 Arm&reg; Compute Library Integration
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+## Building with ACL support
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ACL=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_GRAPH_RUNTIME_ACL=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ACL=ON on the x86 machine and USE_GRAPH_RUNTIME_ACL=ON on the remote AArch64
+device.
+## Usage
+_Note:_ this may not stay up-to-date with changes to the API.
+1. Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+    ```
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides,
+                              layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+    ```
+2. Annotate and partition the graph for ACL.

Review comment:
       Will amend




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOpJSONNode(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvJSONNode(const CallNode* cn);
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);

Review comment:
       I think we don't need this. It is only internally used in the codegen.cc

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOpJSONNode(cn);

Review comment:
       I think we can just do `return JSONSerializer::VisiExpr_(cn);` here

##########
File path: src/runtime/contrib/arm_compute_lib/acl_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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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 ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL 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.";
+    SetupConstants(consts);
+    BuildEngine();
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {

Review comment:
       This one could be moved to base class and it looks to me we don't need to check the format, but instead just return graph_json_

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,76 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,

Review comment:
       MakeRegion

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,76 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,
+                                                                      size_t alignment) {
+  return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(size, alignment);
+}
+
+ACLMemoryRegion::ACLMemoryRegion(size_t size, size_t alignment)
+    : IMemoryRegion(size), ptr_(nullptr) {
+  if (size != 0) {
+    this->ptr_ = this->device_api_->AllocDataSpace(this->ctx_, size, alignment, {});
+  }
+}
+
+ACLMemoryRegion::ACLMemoryRegion(void* ptr, size_t size)
+    : IMemoryRegion(size), ptr_(nullptr), is_subregion_(true) {
+  if (size != 0) {
+    this->ptr_ = ptr;
+  }
+}
+
+ACLMemoryRegion::~ACLMemoryRegion() {
+  if (this->ptr_ != nullptr && !is_subregion_) {
+    this->device_api_->FreeDataSpace(this->ctx_, this->ptr_);
+  }
+}
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLMemoryRegion::extract_subregion(size_t offset,

Review comment:
       ExtractSubRegion

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOpJSONNode(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvJSONNode(const CallNode* cn);
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);

Review comment:
       We don't needs this as well. I believe we don't even needs this file. It looks to me only the header is needed. We can put it in the cc file since no one else is using it.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.h
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/runtime/contrib/arm_compute_lib/acl_allocator.h
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_ARM_COMPUTE_LIB_ACL_ALLOCATOR_H_
+#define TVM_RUNTIME_CONTRIB_ARM_COMPUTE_LIB_ACL_ALLOCATOR_H_
+
+#include <arm_compute/runtime/IAllocator.h>
+#include <arm_compute/runtime/IMemoryRegion.h>
+#include <arm_compute/runtime/MemoryRegion.h>
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+
+#include <memory>
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Override ACL memory allocator and replace with TVM workspace based allocation.
+ */
+class ACLAllocator : public arm_compute::IAllocator {
+ public:
+  ACLAllocator() = default;
+
+  /*!
+   * \brief Allocate bytes to ACL runtime.
+   *
+   * Specific implementation requests memory from TVM using their device api.
+   *
+   * \param size Size to allocate.
+   * \param alignment Alignment that the returned pointer should comply with.
+   * \return A pointer to the allocated memory.
+   */
+  void* allocate(size_t size, size_t alignment) override;
+
+  /*!
+   * \brief Free memory from ACL runtime.
+   *
+   * \param ptr Pointer to workspace to free.
+   */
+  void free(void* ptr) override;
+
+  /*!
+   * \brief Create self-managed memory region.
+   *
+   * \param size Size of the memory region.
+   * \param alignment Alignment of the memory region.
+   * \return The memory region object.
+   */
+  std::unique_ptr<arm_compute::IMemoryRegion> make_region(size_t size, size_t alignment) override;

Review comment:
       MakeRegion

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.h
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/runtime/contrib/arm_compute_lib/acl_allocator.h
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_ARM_COMPUTE_LIB_ACL_ALLOCATOR_H_
+#define TVM_RUNTIME_CONTRIB_ARM_COMPUTE_LIB_ACL_ALLOCATOR_H_
+
+#include <arm_compute/runtime/IAllocator.h>
+#include <arm_compute/runtime/IMemoryRegion.h>
+#include <arm_compute/runtime/MemoryRegion.h>
+#include <tvm/runtime/data_type.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+
+#include <memory>
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Override ACL memory allocator and replace with TVM workspace based allocation.
+ */
+class ACLAllocator : public arm_compute::IAllocator {
+ public:
+  ACLAllocator() = default;
+
+  /*!
+   * \brief Allocate bytes to ACL runtime.
+   *
+   * Specific implementation requests memory from TVM using their device api.
+   *
+   * \param size Size to allocate.
+   * \param alignment Alignment that the returned pointer should comply with.
+   * \return A pointer to the allocated memory.
+   */
+  void* allocate(size_t size, size_t alignment) override;
+
+  /*!
+   * \brief Free memory from ACL runtime.
+   *
+   * \param ptr Pointer to workspace to free.
+   */
+  void free(void* ptr) override;
+
+  /*!
+   * \brief Create self-managed memory region.
+   *
+   * \param size Size of the memory region.
+   * \param alignment Alignment of the memory region.
+   * \return The memory region object.
+   */
+  std::unique_ptr<arm_compute::IMemoryRegion> make_region(size_t size, size_t alignment) override;
+
+ private:
+  /*! \brief Always allocate data in the context of the current CPU. */
+  const TVMContext ctx_{kDLCPU, 0};
+  /*! \brief Device API which allows requests for memory from TVM. */
+  runtime::DeviceAPI* device_api_ = runtime::DeviceAPI::Get(ctx_);
+};
+
+/*!
+ * \brief Memory region that can request TVM memory for ACL to use.
+ */
+class ACLMemoryRegion : public arm_compute::IMemoryRegion {
+ public:
+  ACLMemoryRegion(size_t size, size_t alignment);
+  ACLMemoryRegion(void* ptr, size_t size);
+
+  ~ACLMemoryRegion() override;
+
+  /*! \brief Prevent instances of this class from being copied (As this class contains
+   * pointers). */
+  ACLMemoryRegion(const ACLMemoryRegion&) = delete;
+  /*! \brief Default move constructor. */
+  ACLMemoryRegion(ACLMemoryRegion&&) = default;
+  /*! \brief Prevent instances of this class from being copied (As this class
+   * contains pointers) */
+  ACLMemoryRegion& operator=(const ACLMemoryRegion&) = delete;
+  /*! Default move assignment operator. */
+  ACLMemoryRegion& operator=(ACLMemoryRegion&&) = default;
+
+  void* buffer() override { return this->ptr_; }

Review comment:
       Buffer




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       The reason of having this is the same as Init, because it processes the constants by itself. If we could resolve the constant tensor issue then we don't need to override this function neither.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: tests/python/contrib/test_arm_compute_lib/test_runtime.py
##########
@@ -0,0 +1,98 @@
+# 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.
+"""Arm Compute Library runtime tests."""
+
+import numpy as np
+
+import tvm
+from tvm import relay
+
+from .infrastructure import skip_runtime_test, build_and_run, verify
+from .infrastructure import Device
+
+
+def test_multiple_ops():
+    """
+    Test multiple operators destined for ACL.
+    ACL will expect these ops as in 2 separate functions.

Review comment:
       You're correct, I'll add this to all the tests as ideally this should be checked each 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] FrozenGene commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: tests/python/contrib/test_arm_compute_lib/infrastructure.py
##########
@@ -0,0 +1,197 @@
+# 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.
+from itertools import zip_longest, combinations
+import json
+
+import tvm
+from tvm import relay
+from tvm import rpc
+from tvm.contrib import graph_runtime
+from tvm.relay.op.contrib import arm_compute_lib
+from tvm.contrib import util
+
+
+class Device:
+    """Adjust the following settings to connect to and use a remote device for tests."""
+    use_remote = False
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    # Enable cross compilation when connecting a remote device from a non-arm platform.
+    cross_compile = None
+    # cross_compile = "aarch64-linux-gnu-g++"
+
+    def __init__(self):
+        """Keep remote device for lifetime of object."""
+        self.device = self._get_remote()
+
+    @classmethod
+    def _get_remote(cls):
+        """Get a remote (or local) device to use for testing."""
+        if cls.use_remote:
+            # Here you may adjust settings to run the ACL unit tests via a remote
+            # device using the RPC mechanism. Use this in the case you want to compile
+            # an ACL module on a different machine to what you run the module on i.e.
+            # x86 -> AArch64.
+            #
+            # Use the following to connect directly to a remote device:
+            # device = rpc.connect(
+            #     hostname="0.0.0.0",
+            #     port=9090)
+            #
+            # Or connect via a tracker:
+            # device = tvm.autotvm.measure.request_remote(
+            #     host="0.0.0.0",
+            #     port=9090,
+            #     device_key="device_key",
+            #     timeout=1000)
+            #
+            # return device
+            raise NotImplementedError(
+                "Please adjust these settings to connect to your remote device.")
+        else:
+            device = rpc.LocalSession()
+            return device
+
+
+def get_cpu_op_count(mod):
+    """Traverse graph counting ops offloaded to TVM."""
+    class Counter(tvm.relay.ExprVisitor):
+        def __init__(self):
+            super().__init__()
+            self.count = 0
+
+        def visit_call(self, call):
+            if isinstance(call.op, tvm.ir.Op):
+                self.count += 1
+
+            super().visit_call(call)
+
+    c = Counter()
+    c.visit(mod["main"])
+    return c.count
+
+
+def skip_runtime_test():
+    """Skip test if it requires the runtime and it's not present."""
+    # ACL codegen not present.
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+    # Remote device is in use or ACL runtime not present
+    if not Device.use_remote and not arm_compute_lib.is_arm_compute_runtime_enabled():
+        print("Skip because runtime isn't present or a remote device isn't being used.")
+        return True
+
+
+def skip_codegen_test():
+    """Skip test if it requires the ACL codegen and it's not present."""
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+
+def build_module(mod, target, params=None, enable_acl=True, tvm_ops=0, acl_partitions=1):
+    """Build module with option to build for ACL."""
+    if isinstance(mod, tvm.relay.expr.Call):
+        mod = tvm.IRModule.from_expr(mod)
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        if enable_acl:
+            mod = arm_compute_lib.partition_for_arm_compute_lib(mod, params)
+            tvm_op_count = get_cpu_op_count(mod)
+            assert tvm_op_count == tvm_ops, \
+                "Got {} TVM operators, expected {}".format(tvm_op_count, tvm_ops)
+            partition_count = 0
+            for global_var in mod.get_global_vars():
+                if "arm_compute_lib" in global_var.name_hint:
+                    partition_count += 1
+
+            assert acl_partitions == partition_count, \
+                "Got {} Arm Compute Library partitions, expected {}".format(
+                    partition_count, acl_partitions)
+        relay.backend.compile_engine.get().clear()
+        return relay.build(mod, target=target, params=params)
+
+
+def build_and_run(mod, inputs, outputs, params, device, enable_acl=True, no_runs=1,
+                  tvm_ops=0, acl_partitions=1):
+    """Build and run the relay module."""
+    lib = build_module(mod, device.target, params, enable_acl, tvm_ops, acl_partitions)
+    lib = update_lib(lib, device.device, device.cross_compile)
+    gen_module = graph_runtime.GraphModule(lib['default'](device.device.cpu(0)))
+    gen_module.set_input(**inputs)
+    out = []
+    for _ in range(no_runs):
+        gen_module.run()
+        out.append([gen_module.get_output(i) for i in range(outputs)])
+    return out
+
+
+def update_lib(lib, device, cross_compile):
+    """Export the library to the remote/local device."""
+    lib_name = "mod.so"
+    temp = util.tempdir()
+    lib_path = temp.relpath(lib_name)
+    if cross_compile:
+        lib.export_library(lib_path, cc=cross_compile)
+    else:
+        lib.export_library(lib_path)
+    device.upload(lib_path)
+    lib = device.load_module(lib_name)
+    return lib
+
+
+def verify(answers, atol, rtol):
+    """Compare the array of answers. Each entry is a list of outputs."""
+    if len(answers) < 2:
+        raise RuntimeError(
+            f"No results to compare: expected at least two, found {len(answers)}")
+    for answer in zip_longest(*answers):
+        for outs in combinations(answer, 2):
+            tvm.testing.assert_allclose(
+               outs[0].asnumpy(), outs[1].asnumpy(), rtol=rtol, atol=atol)
+
+
+def extract_acl_modules(module):
+    """Get the ACL module(s) from llvm module."""
+    return list(filter(lambda mod: mod.type_key == "arm_compute_lib",
+                       module.lib.imported_modules))

Review comment:
       Let us add one function `get_lib` function instead of using attribute directly inside `GraphRuntimeFactoryModule` if it is a must.

##########
File path: tests/python/contrib/test_arm_compute_lib/infrastructure.py
##########
@@ -0,0 +1,197 @@
+# 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.
+from itertools import zip_longest, combinations
+import json
+
+import tvm
+from tvm import relay
+from tvm import rpc
+from tvm.contrib import graph_runtime
+from tvm.relay.op.contrib import arm_compute_lib
+from tvm.contrib import util
+
+
+class Device:
+    """Adjust the following settings to connect to and use a remote device for tests."""
+    use_remote = False
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    # Enable cross compilation when connecting a remote device from a non-arm platform.
+    cross_compile = None
+    # cross_compile = "aarch64-linux-gnu-g++"
+
+    def __init__(self):
+        """Keep remote device for lifetime of object."""
+        self.device = self._get_remote()
+
+    @classmethod
+    def _get_remote(cls):
+        """Get a remote (or local) device to use for testing."""
+        if cls.use_remote:
+            # Here you may adjust settings to run the ACL unit tests via a remote
+            # device using the RPC mechanism. Use this in the case you want to compile
+            # an ACL module on a different machine to what you run the module on i.e.
+            # x86 -> AArch64.
+            #
+            # Use the following to connect directly to a remote device:
+            # device = rpc.connect(
+            #     hostname="0.0.0.0",
+            #     port=9090)
+            #
+            # Or connect via a tracker:
+            # device = tvm.autotvm.measure.request_remote(
+            #     host="0.0.0.0",
+            #     port=9090,
+            #     device_key="device_key",
+            #     timeout=1000)
+            #
+            # return device
+            raise NotImplementedError(
+                "Please adjust these settings to connect to your remote device.")
+        else:
+            device = rpc.LocalSession()
+            return device
+
+
+def get_cpu_op_count(mod):
+    """Traverse graph counting ops offloaded to TVM."""
+    class Counter(tvm.relay.ExprVisitor):
+        def __init__(self):
+            super().__init__()
+            self.count = 0
+
+        def visit_call(self, call):
+            if isinstance(call.op, tvm.ir.Op):
+                self.count += 1
+
+            super().visit_call(call)
+
+    c = Counter()
+    c.visit(mod["main"])
+    return c.count
+
+
+def skip_runtime_test():
+    """Skip test if it requires the runtime and it's not present."""
+    # ACL codegen not present.
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+    # Remote device is in use or ACL runtime not present
+    if not Device.use_remote and not arm_compute_lib.is_arm_compute_runtime_enabled():
+        print("Skip because runtime isn't present or a remote device isn't being used.")
+        return True
+
+
+def skip_codegen_test():
+    """Skip test if it requires the ACL codegen and it's not present."""
+    if not tvm.get_global_func("relay.ext.arm_compute_lib", True):
+        print("Skip because Arm Compute Library codegen is not available.")
+        return True
+
+
+def build_module(mod, target, params=None, enable_acl=True, tvm_ops=0, acl_partitions=1):
+    """Build module with option to build for ACL."""
+    if isinstance(mod, tvm.relay.expr.Call):
+        mod = tvm.IRModule.from_expr(mod)
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        if enable_acl:
+            mod = arm_compute_lib.partition_for_arm_compute_lib(mod, params)
+            tvm_op_count = get_cpu_op_count(mod)
+            assert tvm_op_count == tvm_ops, \
+                "Got {} TVM operators, expected {}".format(tvm_op_count, tvm_ops)
+            partition_count = 0
+            for global_var in mod.get_global_vars():
+                if "arm_compute_lib" in global_var.name_hint:
+                    partition_count += 1
+
+            assert acl_partitions == partition_count, \
+                "Got {} Arm Compute Library partitions, expected {}".format(
+                    partition_count, acl_partitions)
+        relay.backend.compile_engine.get().clear()
+        return relay.build(mod, target=target, params=params)
+
+
+def build_and_run(mod, inputs, outputs, params, device, enable_acl=True, no_runs=1,
+                  tvm_ops=0, acl_partitions=1):
+    """Build and run the relay module."""
+    lib = build_module(mod, device.target, params, enable_acl, tvm_ops, acl_partitions)
+    lib = update_lib(lib, device.device, device.cross_compile)
+    gen_module = graph_runtime.GraphModule(lib['default'](device.device.cpu(0)))
+    gen_module.set_input(**inputs)
+    out = []
+    for _ in range(no_runs):
+        gen_module.run()
+        out.append([gen_module.get_output(i) for i in range(outputs)])
+    return out
+
+
+def update_lib(lib, device, cross_compile):
+    """Export the library to the remote/local device."""
+    lib_name = "mod.so"
+    temp = util.tempdir()
+    lib_path = temp.relpath(lib_name)
+    if cross_compile:
+        lib.export_library(lib_path, cc=cross_compile)
+    else:
+        lib.export_library(lib_path)
+    device.upload(lib_path)
+    lib = device.load_module(lib_name)
+    return lib
+
+
+def verify(answers, atol, rtol):
+    """Compare the array of answers. Each entry is a list of outputs."""
+    if len(answers) < 2:
+        raise RuntimeError(
+            f"No results to compare: expected at least two, found {len(answers)}")
+    for answer in zip_longest(*answers):
+        for outs in combinations(answer, 2):
+            tvm.testing.assert_allclose(
+               outs[0].asnumpy(), outs[1].asnumpy(), rtol=rtol, atol=atol)
+
+
+def extract_acl_modules(module):
+    """Get the ACL module(s) from llvm module."""
+    return list(filter(lambda mod: mod.type_key == "arm_compute_lib",
+                       module.lib.imported_modules))

Review comment:
       Let us add one function `get_lib` instead of using attribute directly inside `GraphRuntimeFactoryModule` if it is a must.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   Not too sure what happened to the Mac build, looks unrelated.


----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""Arm Compute Library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_enabled():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    return tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled")()

Review comment:
       If users didn't build with ACL and call this function, then it will throw a ValueError. You might still need a checker to test if the global function `relay.op.is_arm_compute_runtime_enabled` is registered.

##########
File path: tests/python/contrib/test_arm_compute_lib/test_runtime.py
##########
@@ -0,0 +1,98 @@
+# 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.
+"""Arm Compute Library runtime tests."""
+
+import numpy as np
+
+import tvm
+from tvm import relay
+
+from .infrastructure import skip_runtime_test, build_and_run, verify
+from .infrastructure import Device
+
+
+def test_multiple_ops():
+    """
+    Test multiple operators destined for ACL.
+    ACL will expect these ops as in 2 separate functions.

Review comment:
       It seems to me that this test doesn't really test the number of ACL functions directly. For example, if ACL runtime supports multiple kernel ops in the future, then this test will still pass even if the partitioning result is unexpected. I would suggest adding tests for `partition_for_arm_compute_lib` to check if the transformed Relay graph matches our expectation, including composite functions and partitioned ACL functions.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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 ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL 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.";
+    SetupConstants(consts);
+    BuildEngine();
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {
+    if (format == "json") {
+      return graph_json_;
+    }
+    LOG(FATAL) << "Format not supported by Arm Compute Library runtime.";
+    return "";
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      uint32_t eid = EntryID(nid, 0);
+      if (nodes_[nid].GetOpType() == "input") {
+        void* data = data_entry_[eid]->data;
+        CheckACLError(layer_.inputs[i].allocator()->import_memory(data));
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      void* data = data_entry_[eid]->data;
+      CheckACLError(layer_.outputs[i].allocator()->import_memory(data));
+    }
+
+    this->layer_.function->run();
+  }
+
+ private:
+  /*!
+   * \brief Build ACL layer from JSON representation and cache.
+   *
+   * \note For the time being only one layer or operator is supported
+   * per engine.
+   */
+  void BuildEngine() {
+    std::shared_ptr<arm_compute::MemoryManagerOnDemand> mm = MakeMemoryManager();
+    int num_pools = 0;
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      uint32_t nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "input") {
+        layer_.inputs.push_back(MakeTensor(node));
+      } else if (node.GetOpType() == "const") {
+        uint32_t eid = EntryID(nid, 0);
+        void* data = data_entry_[eid]->data;
+        layer_.const_inputs.push_back(MakeTensor(node, data));
+      }
+    }
+
+    bool found_kernel_node = false;
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (found_kernel_node) {
+        LOG(FATAL)
+            << "Arm Compute Library runtime module only supports one kernel node per function.";
+      }
+      if (node.GetOpType() == "kernel") {
+        found_kernel_node = true;
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name || "arm_compute_lib.conv2d" == op_name) {
+          CreateConvolution2DLayer(&layer_, node, mm);
+          num_pools++;
+        } else if ("nn.max_pool2d" == op_name) {
+          CreatePoolingLayer(&layer_, node);
+        } else if ("reshape" == op_name) {
+          CreateReshapeLayer(&layer_, node);
+        } else {
+          LOG(FATAL) << "Unsupported op: " << op_name;
+        }
+      }
+    }
+
+    this->layer_.function->prepare();
+    if (num_pools > 0) mm->populate(this->allocator_, num_pools);
+  }
+
+  /*!
+   * \brief ACL objects we cache in order to avoid needing to construct
+   * a new layer each time.
+   */
+  struct CachedLayer {
+    std::shared_ptr<arm_compute::IFunction> function;
+    std::vector<arm_compute::Tensor> inputs;
+    std::vector<arm_compute::Tensor> const_inputs;
+    std::vector<arm_compute::Tensor> outputs;
+  };
+
+  /*!
+   * \brief Create a 2D convolution layer.
+   *
+   * \param layer The ACL layer to build. Containing inputs, outputs and the ACL function.
+   * \param node The JSON representation of the operator.
+   * \param mm The ACL conv2d layer can request auxiliary memory from TVM.
+   */
+  static void CreateConvolution2DLayer(
+      CachedLayer* layer, const JSONGraphNode& node,
+      const std::shared_ptr<arm_compute::MemoryManagerOnDemand>& mm) {
+    std::vector<std::string> padding = node.GetAttr<std::vector<std::string>>("padding");
+    std::vector<std::string> strides = node.GetAttr<std::vector<std::string>>("strides");
+    std::vector<std::string> dilation = node.GetAttr<std::vector<std::string>>("dilation");
+    arm_compute::PadStrideInfo pad_stride_info = ToACLPadStride(padding, strides);
+
+    int groups = std::stoi(node.GetAttr<std::vector<std::string>>("groups")[0]);
+    CHECK(groups == 1) << "Arm Compute Library NEON convolution only supports group size of 1.";
+
+    arm_compute::ActivationLayerInfo act_info;
+    if (node.HasAttr("activation_type")) {
+      std::string activation_type = node.GetAttr<std::vector<std::string>>("activation_type")[0];
+      if (activation_type == "relu") {
+        act_info = arm_compute::ActivationLayerInfo(
+            arm_compute::ActivationLayerInfo::ActivationFunction::RELU);
+      } else {
+        LOG(FATAL) << "Unsupported activation function";
+      }
+    }
+
+    arm_compute::Size2D dilation_2d(std::stoi(dilation[0]), std::stoi(dilation[1]));
+
+    layer->outputs.push_back(MakeOutputTensor(node.GetOpShape()[0]));
+
+    auto function = std::make_shared<arm_compute::NEConvolutionLayer>(mm);
+    function->configure(&layer->inputs[0], &layer->const_inputs[0],
+                        layer->const_inputs.size() > 1 ? &layer->const_inputs[1] : nullptr,
+                        &layer->outputs[0], pad_stride_info, arm_compute::WeightsInfo(),
+                        dilation_2d, act_info);
+    layer->function = function;
+  }
+
+  /*!
+   * \brief Create a pooling layer.
+   *
+   * \note Currently only maxpool is supported.
+   *
+   * \param layer The ACL layer to build. Containing inputs, outputs and the ACL function.
+   * \param node The JSON representation of the operator.
+   */
+  static void CreatePoolingLayer(CachedLayer* layer, const JSONGraphNode& node) {
+    std::vector<std::string> padding = node.GetAttr<std::vector<std::string>>("padding");
+    std::vector<std::string> strides = node.GetAttr<std::vector<std::string>>("strides");
+    arm_compute::PadStrideInfo pad_stride_info = ToACLPadStride(padding, strides);
+
+    auto attr_pool_size = node.GetAttr<std::vector<std::string>>("pool_size");
+    int pool_size_h = std::stoi(attr_pool_size[0]);
+    int pool_size_w = std::stoi(attr_pool_size[1]);
+
+    arm_compute::PoolingType pool_type;
+    if (node.GetOpName() == "nn.max_pool2d") {
+      pool_type = arm_compute::PoolingType::MAX;
+    } else {
+      LOG(FATAL) << "Pooling type not supported";
+    }
+
+    arm_compute::PoolingLayerInfo pool_info =
+        arm_compute::PoolingLayerInfo(pool_type, arm_compute::Size2D(pool_size_h, pool_size_w),
+                                      arm_compute::DataLayout::NHWC, pad_stride_info);
+
+    layer->outputs.push_back(MakeOutputTensor(node.GetOpShape()[0]));
+
+    auto function = std::make_shared<arm_compute::NEPoolingLayer>();
+    function->configure(&layer->inputs[0], &layer->outputs[0], pool_info);
+    layer->function = function;
+  }
+
+  /*!
+   * \brief Create a reshape layer.
+   *
+   * \param layer The ACL layer to build. Containing inputs, outputs and the ACL function.
+   * \param node The JSON representation of the operator.
+   */
+  static void CreateReshapeLayer(CachedLayer* layer, const JSONGraphNode& node) {
+    layer->outputs.push_back(MakeOutputTensor(node.GetOpShape()[0]));
+    auto function = std::make_shared<arm_compute::NEReshapeLayer>();
+    function->configure(&layer->inputs[0], &layer->outputs[0]);
+    layer->function = function;
+  }
+
+  /*! \brief Allow ACL functions to request auxiliary memory from TVM. */
+  arm_compute_lib::ACLAllocator allocator_;
+  /*!
+   * \brief The network layers represented by acl functions.
+   * \note Currently only supports a single layer.
+   */
+  CachedLayer layer_;
+#else
+  void Run() override {
+    LOG(FATAL) << "Cannot call run on Arm Compute Library module without runtime enabled. "
+               << "Please build with USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME.";
+  }
+
+  void BuildEngine() {
+    // Do nothing.

Review comment:
       Maybe we could have a warning `LOG(WARNING)` to mention the engine is actually not initialized.

##########
File path: tests/python/contrib/test_arm_compute_lib/test_runtime.py
##########
@@ -0,0 +1,98 @@
+# 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.
+"""Arm Compute Library runtime tests."""
+
+import numpy as np
+
+import tvm
+from tvm import relay
+
+from .infrastructure import skip_runtime_test, build_and_run, verify
+from .infrastructure import Device
+
+
+def test_multiple_ops():
+    """
+    Test multiple operators destined for ACL.
+    ACL will expect these ops as in 2 separate functions.
+    """
+    if skip_runtime_test():
+        return
+
+    device = Device()
+    np.random.seed(0)
+
+    def get_model(input_shape, var_names):
+        """Return a model and any parameters it may have."""
+        a = relay.var(next(var_names), shape=input_shape, dtype="float32")
+        out = relay.reshape(a, (1, 1, 1000))
+        out = relay.reshape(out, (1, 1000))
+        return out
+
+    inputs = {
+        "a": tvm.nd.array(np.random.uniform(0, 1, (1, 1, 1, 1000)).astype("float32"))
+    }
+
+    outputs = []
+    for acl in [False, True]:
+        func = get_model(inputs["a"].shape, iter(inputs))
+        outputs.append(build_and_run(func, inputs, 1, None, device,
+                                     enable_acl=acl))
+    verify(outputs, atol=0.002, rtol=0.01)
+
+
+def test_multiple_runs():
+    """
+    Test that multiple runs of an operator work.
+    Note: the result isn't checked.

Review comment:
       Why not checking the results?




----------------------------------------------------------------
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] FrozenGene commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,138 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ARM_COMPUTE_LIB=ON on the x86 machine and USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON on the remote
+AArch64 device.
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)

Review comment:
       I think we miss the mutated new module? That is we should have `module = partition_for_arm_compute_lib(module)`?

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,74 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,
+                                                                      size_t alignment) {
+  return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(size, alignment);

Review comment:
       Should be `std::make_unique`

##########
File path: cmake/config.cmake
##########
@@ -184,6 +184,18 @@ set(USE_SORT ON)
 # Whether use MKL-DNN (DNNL) codegen
 set(USE_DNNL_CODEGEN OFF)
 
+# Whether to use Arm Compute Library (ACL) codegen
+# We provide 2 separate flags since we cannot build the ACL runtime on x86.
+# This is useful for cases where you want to cross-compile a relay graph
+# on x86 then run on AArch.

Review comment:
       Could we list an example how to cross compile on x86 but run AArch use these two flags? As we have done it in the tutorial doc, let us copy it here too.

##########
File path: docs/deploy/arm_compute_lib.rst
##########
@@ -0,0 +1,138 @@
+..  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 Arm|reg| Compute Library Integration
+==========================================
+
+Introduction
+------------
+
+Arm Compute Library (ACL) is an open source project that provides accelerated kernels for Arm CPU's
+and GPU's. Currently the integration offloads operators to ACL to use hand-crafted assembler
+routines in the library. By offloading select operators from a relay graph to ACL we can achieve
+a performance boost on such devices.
+
+Building with ACL support
+-------------------------
+
+The current implementation has two separate build options in cmake. The reason for this split is
+because ACL cannot be used on an x86 machine. However, we still want to be able compile an ACL
+runtime module on an x86 machine.
+
+* USE_ARM_COMPUTE_LIB=ON/OFF - Enabling this flag will add support for compiling an ACL runtime module.
+* USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON/OFF/path-to-acl - Enabling this flag will allow the graph runtime to
+  compute the ACL offloaded functions.
+
+These flags can be used in different scenarios depending on your setup. For example, if you want
+to compile ACL on an x86 machine and then run the module on a remote Arm device via RPC, you will
+need to use USE_ARM_COMPUTE_LIB=ON on the x86 machine and USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME=ON on the remote
+AArch64 device.
+
+Usage
+-----
+
+.. note::
+
+    This section may not stay up-to-date with changes to the API.
+
+Create a relay graph. This may be a single operator or a whole graph. The intention is that any
+relay graph can be input. The ACL integration will only pick supported operators to be offloaded
+whilst the rest will be computed via TVM. (For this example we will use a single
+max_pool2d operator).
+
+.. code:: python
+
+    import tvm
+    from tvm import relay
+
+    data_type = "float32"
+    data_shape = (1, 14, 14, 512)
+    strides = (2, 2)
+    padding = (0, 0, 0, 0)
+    pool_size = (2, 2)
+    layout = "NHWC"
+    output_shape = (1, 7, 7, 512)
+
+    data = relay.var('data', shape=data_shape, dtype=data_type)
+    out = relay.nn.max_pool2d(data, pool_size=pool_size, strides=strides, layout=layout, padding=padding)
+    module = tvm.IRModule.from_expr(out)
+
+
+Annotate and partition the graph for ACL.
+
+..code:: python
+
+    from tvm.relay.op.contrib.arm_compute_lib import partition_for_arm_compute_lib
+    partition_for_arm_compute_lib(module)
+
+
+Build the Relay graph.
+
+.. code:: python
+
+    target = "llvm -mtriple=aarch64-linux-gnu -mattr=+neon"
+    with tvm.transform.PassContext(opt_level=3, disabled_pass=["AlterOpLayout"]):
+        lib = relay.build(module, target=target)
+
+
+Export the module.
+
+.. code:: python
+
+    lib_path = '~/lib_acl.so'
+    cross_compile = 'aarch64-linux-gnu-c++'
+    lib.export_library(lib_path, cc=cross_compile)
+
+
+Run Inference. This must be on an Arm device. If compiling on x86 device and running on aarch64,
+consider using the RPC mechanism.

Review comment:
       Let us add a doc link of RPC: https://tvm.apache.org/docs/tutorials/cross_compilation_and_rpc.html#sphx-glr-tutorials-cross-compilation-and-rpc-py




----------------------------------------------------------------
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] manupa-arm commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #5915:
URL: https://github.com/apache/incubator-tvm/pull/5915#discussion_r456314032



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOpJSONNode(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";

Review comment:
       Minor comment : Is there a downside going into nested function hierarchies that are not composite functions. What if just use JSONSerializer::VisiExpr_(cn) for every other case if its not a composite function?

##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_enabled():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    return tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.MergeComposite(arm_compute_lib_pattern_table()),
+                                    transform.AnnotateTarget('arm_compute_lib'),
+                                    transform.PartitionGraph()])
+
+    return seq(mod)
+
+
+@register_pattern_table("arm_compute_lib")
+def arm_compute_lib_pattern_table():
+    """Get the ACL pattern table."""
+
+    def conv_pattern():
+        """Create a convolution pattern.
+
+        Returns
+        -------
+        pattern : dataflow_pattern.AltPattern
+            Denotes the convolution pattern.
+        """
+        pattern = is_op('nn.pad')(wildcard()) | wildcard()
+        pattern = is_op('nn.conv2d')(pattern, is_constant())
+        pattern = pattern.optional(lambda x: is_op('nn.bias_add')(x, is_constant()))
+        pattern = pattern.optional(is_op('nn.relu'))

Review comment:
       Aren't all Relu variants are lowered to clip in Relay? Thus, not sure this one will ever get matched?




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOpJSONNode(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvJSONNode(const CallNode* cn);
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);

Review comment:
       Will change, for me it was just about readability




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }

Review comment:
       This was a concern of mine too, although I couldn't find a way around this without looking more into MetadataModule.




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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


   let us consider expand the name acl to arm_compute_lib or some other alternatives, since ACL means different things to ML/NLP audiences


----------------------------------------------------------------
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] FrozenGene commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,220 @@
+/*
+ * 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/arm_compute_lib/codegen.cc
+ * \brief Implementation of the Relay -> ACL 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"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  /*!
+   * \brief Visit call nodes and generate appropriate JSON node.
+   *
+   * \param cn The current call node.
+   * \return A list of graph entry nodes.
+   */
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override {
+    if (cn->op.as<OpNode>()) {
+      return JSONSerializer::VisitExpr_(cn);
+    }
+    if (!cn->op.as<FunctionNode>()) {
+      LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+                 << cn->op->GetTypeKey();
+    }
+    auto fn = cn->op.as<FunctionNode>();
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    const std::string name = comp.value();
+    std::shared_ptr<JSONGraphNode> json_node;
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvJSONNode(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+    return AddNode(json_node, GetRef<Expr>(cn));
+  }
+
+ private:
+  /*!
+   * \brief Create a JSON representation of a composite convolution.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvJSONNode(const CallNode* cn) {
+    const std::string name = "nn.conv2d";
+    const CallNode* pad = nullptr;
+    const CallNode* conv;

Review comment:
       Nitpicking comment: let us add `nullptr` to const CallNode* conv = nullptr;




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,119 @@
+# 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
+"""ACL library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_enabled():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    return tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.MergeComposite(arm_compute_lib_pattern_table()),
+                                    transform.AnnotateTarget('arm_compute_lib'),
+                                    transform.PartitionGraph()])
+
+    return seq(mod)
+
+
+@register_pattern_table("arm_compute_lib")
+def arm_compute_lib_pattern_table():
+    """Get the ACL pattern table."""
+
+    def conv_pattern():
+        """Create a convolution pattern.
+
+        Returns
+        -------
+        pattern : dataflow_pattern.AltPattern
+            Denotes the convolution pattern.
+        """
+        pattern = is_op('nn.pad')(wildcard()) | wildcard()
+        pattern = is_op('nn.conv2d')(pattern, is_constant())
+        pattern = pattern.optional(lambda x: is_op('nn.bias_add')(x, is_constant()))
+        pattern = pattern.optional(is_op('nn.relu'))

Review comment:
       Yes it can, there are a number of fused activations that can be supported, although to keep this PR simple I left it out for the time being. 

##########
File path: src/runtime/contrib/arm_compute_lib/acl_allocator.cc
##########
@@ -0,0 +1,77 @@
+/*
+ * 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_allocator.cc
+ * \brief ACL Allocator implementation that requests memory from TVM.
+ */
+
+#include "acl_allocator.h"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+namespace arm_compute_lib {
+
+void* ACLAllocator::allocate(size_t size, size_t alignment) {
+  CHECK_GT(size, 0) << "Cannot allocate size less than or equal to zero";
+  return this->device_api_->AllocWorkspace(this->ctx_, size, {});
+}
+
+void ACLAllocator::free(void* ptr) { this->device_api_->FreeWorkspace(this->ctx_, ptr); }
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLAllocator::make_region(size_t size,
+                                                                      size_t alignment) {
+  return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(size, alignment);
+}
+
+ACLMemoryRegion::ACLMemoryRegion(size_t size, size_t alignment)
+    : IMemoryRegion(size), ptr_(nullptr) {
+  if (size != 0) {
+    this->ptr_ = this->device_api_->AllocDataSpace(this->ctx_, size, alignment, {});
+  }
+}
+
+ACLMemoryRegion::ACLMemoryRegion(void* ptr, size_t size)
+    : IMemoryRegion(size), ptr_(nullptr), is_subregion_(true) {
+  if (size != 0) {
+    this->ptr_ = ptr;
+  }
+}
+
+ACLMemoryRegion::~ACLMemoryRegion() {
+  if (this->ptr_ != nullptr && !is_subregion_) {
+    this->device_api_->FreeDataSpace(this->ctx_, this->ptr_);
+  }
+}
+
+std::unique_ptr<arm_compute::IMemoryRegion> ACLMemoryRegion::extract_subregion(size_t offset,
+                                                                               size_t size) {
+  if (this->ptr_ != nullptr && (offset < _size) && (_size - offset >= size)) {
+    return arm_compute::support::cpp14::make_unique<ACLMemoryRegion>(

Review comment:
       Thanks for the catch




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,143 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const ConstantNode* cn) override;
+
+  /*!
+   * \brief Get the constant data transposed when pre-processing the
+   * input function.
+   *
+   * \return An array of constants
+   */
+  Array<runtime::NDArray> GetParamsData();
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOp(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvolution(const CallNode* cn);
+
+  /* \brief Transposed constant tensors to serialize. Arm Compute Library expects constant tensors
+   * in OHWI format. */
+  Array<runtime::NDArray> constants_;
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only currently supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);
+
+/*!
+ * \brief Get the external symbol of the Relay function name.
+ *
+ * \param func The provided function.
+ *
+ * \return An external symbol.
+ */
+std::string GetExtSymbol(const Function& func) {
+  const auto name_node = func->GetAttr<String>(tvm::attr::kGlobalSymbol);
+  CHECK(name_node.defined()) << "Fail to retrieve external symbol.";
+  return std::string(name_node.value());
+}
+
+TVM_REGISTER_GLOBAL("relay.ext.arm_compute_lib").set_body_typed(ACLCompiler);

Review comment:
       put this in the cc file

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,143 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const ConstantNode* cn) override;
+
+  /*!
+   * \brief Get the constant data transposed when pre-processing the
+   * input function.
+   *
+   * \return An array of constants
+   */
+  Array<runtime::NDArray> GetParamsData();
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOp(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvolution(const CallNode* cn);
+
+  /* \brief Transposed constant tensors to serialize. Arm Compute Library expects constant tensors
+   * in OHWI format. */
+  Array<runtime::NDArray> constants_;
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only currently supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);
+
+/*!
+ * \brief Get the external symbol of the Relay function name.
+ *
+ * \param func The provided function.
+ *
+ * \return An external symbol.
+ */
+std::string GetExtSymbol(const Function& func) {

Review comment:
       yeah, we should.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }
+
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB

Review comment:
       As commented above, I think we don't need this guard. We can simply not compile this file and all other ACL related files when `USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME` is not enabled. Please correct me if I miss something.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        this->Init();
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  /*!
+   * \brief Save a compiled network to a binary stream, which can then be
+   * serialized to disk.
+   *
+   * \param stream The stream to save the binary.
+   */
+  void SaveToBinary(dmlc::Stream* stream) override {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> const_names;
+    for (const auto& it : const_names_) {
+      const_names.push_back(it);
+    }
+    stream->Write(const_names);
+    // Save the required constant data
+    stream->Write(constants_.size());
+    for (const auto& it : constants_) {
+      it.Save(stream);
+    }
+  }
+
+  /*!
+   * \brief Load a compiled network from stream.
+   *
+   * \param strm The binary stream to load.
+   * \return The created ACL module.
+   */
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    size_t const_data_count;
+    CHECK(stream->Read(&const_data_count));
+    Array<NDArray> const_data;
+    for (size_t i = 0; i < const_data_count; ++i) {
+      runtime::NDArray temp;
+      CHECK(temp.Load(stream)) << "Failed to load constant";
+      const_data.push_back(temp);
+    }
+    auto n = make_object<ACLRuntime>(symbol, graph_json, const_names, const_data);
+    return Module(n);
+  }
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   */
+  void Init() {
+    CHECK_EQ(this->constants_.size(), const_idx_.size())
+        << "The number of input constants must match the number expected.";
+    this->SetupConstants(this->constants_);
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+    BuildEngine();
+#endif
+  }
+
+  // Do not accept constants from MetadataModule as they should be transposed
+  // by the ACL codegen so they have the correct expected layout.
+  void Init(const Array<NDArray>& constants) override { LOG(FATAL) << "Not implemented."; }

Review comment:
       This is because the Function passed to the codegen is updated. Let's think a better way to keep the updated Function so that constants can be correctly collected and all the logic here can follow what dnnl json runtime does exactly. Otherwise, multiple copies of constants are saved which leads to very high memory consumption.

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+
+/*!
+ * \brief ACL objects we cache in order to avoid needing to construct
+ * a new layer each time.
+ */
+struct CachedLayer {
+  std::shared_ptr<arm_compute::IFunction> function;
+  std::vector<arm_compute::Tensor> inputs;
+  std::vector<arm_compute::Tensor> const_inputs;
+  std::vector<arm_compute::Tensor> outputs;
+};
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL 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.
+   * \params consts An array of constants pre-transposed to the correct layout expected by ACL.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names, const Array<NDArray>& consts)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {
+    this->constants_ = consts;
+  }
+
+  /*!
+   * \brief Get a packed function.
+   *
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       Do we need this? They are in the base class already?

##########
File path: cmake/modules/contrib/ArmComputeLib.cmake
##########
@@ -0,0 +1,66 @@
+# 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.
+
+# We separate the codegen and runtime build since ACL can only be built
+# for AArch. In the world where we take the cross compilation approach,
+# which is common with arm devices, we need to be able to cross-compile
+# a relay graph on x86 for AArch and then run the graph on AArch.
+if(USE_ARM_COMPUTE_LIB)
+    file(GLOB ACL_RELAY_CONTRIB_SRC src/relay/backend/contrib/arm_compute_lib/*.cc)
+    file(GLOB ACL_RUNTIME_MODULE src/runtime/contrib/arm_compute_lib/acl_runtime.cc)
+    list(APPEND COMPILER_SRCS ${ACL_RELAY_CONTRIB_SRC})
+    list(APPEND COMPILER_SRCS ${ACL_RUNTIME_MODULE})
+    message(STATUS "Build with Arm Compute Library support...")
+endif()
+
+if(USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME)
+    set(ACL_PATH ${CMAKE_CURRENT_SOURCE_DIR}/acl)
+    # Detect custom ACL path.
+    if (NOT USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME STREQUAL "ON")
+        set(ACL_PATH ${USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME})
+    endif()
+
+    file(GLOB ACL_CONTRIB_SRC src/runtime/contrib/arm_compute_lib/*)
+
+    set(ACL_INCLUDE_DIRS ${ACL_PATH}/include ${ACL_PATH})
+    include_directories(${ACL_INCLUDE_DIRS})
+
+    find_library(EXTERN_ACL_COMPUTE_LIB
+            NAMES arm_compute libarm_compute
+            HINTS "${ACL_PATH}" "${ACL_PATH}/lib" "${ACL_PATH}/build"
+            )
+    find_library(EXTERN_ACL_COMPUTE_CORE_LIB
+            NAMES arm_compute_core libarm_compute_core
+            HINTS "${ACL_PATH}" "${ACL_PATH}/lib" "${ACL_PATH}/build"
+            )
+    find_library(EXTERN_ACL_COMPUTE_GRAPH_LIB
+            NAMES arm_compute_graph libarm_compute_graph
+            HINTS "${ACL_PATH}" "${ACL_PATH}/lib" "${ACL_PATH}/build"
+            )
+
+    list(APPEND TVM_RUNTIME_LINKER_LIBS ${EXTERN_ACL_COMPUTE_LIB})
+    list(APPEND TVM_RUNTIME_LINKER_LIBS ${EXTERN_ACL_COMPUTE_CORE_LIB})
+    list(APPEND TVM_RUNTIME_LINKER_LIBS ${EXTERN_ACL_COMPUTE_GRAPH_LIB})
+    list(APPEND RUNTIME_SRCS ${ACL_CONTRIB_SRC})
+    message(STATUS "Build with Arm Compute Library graph runtime support: "
+            ${EXTERN_ACL_COMPUTE_LIB} ", \n"
+            ${EXTERN_ACL_COMPUTE_CORE_LIB} ", \n"
+            ${EXTERN_ACL_COMPUTE_GRAPH_LIB})
+
+    # Set flag to detect ACL graph runtime support.
+    add_definitions(-DTVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB)

Review comment:
       Do we need this? Shouldn't we enable it directly when `USE_ARM_COMPUTE_LIB_GRAPH_RUNTIME` is already enabled.

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen_acl.h
##########
@@ -0,0 +1,143 @@
+/*
+ * 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/arm_compute_lib/codegen_acl.h
+ * \brief The Relay -> ACL JSON schema compiler.
+ */
+
+#ifndef TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+#define TVM_RELAY_BACKEND_CONTRIB_ARM_COMPUTE_LIB_CODEGEN_ACL_H_
+
+#include <tvm/relay/expr_functor.h>
+
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../codegen_json/codegen_json.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+/*!
+ * \brief Generates an ACLModule from a relay expression. This "compilation"
+ * does not require ACL since the actual conversion using ACL APIs is
+ * deferred until creation of the runtime. This step simply serializes the
+ * relay program into a JSON string.
+ */
+class ACLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  ACLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override;
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const ConstantNode* cn) override;
+
+  /*!
+   * \brief Get the constant data transposed when pre-processing the
+   * input function.
+   *
+   * \return An array of constants
+   */
+  Array<runtime::NDArray> GetParamsData();
+
+ private:
+  /*!
+   * \brief Create a JSON representation of an operator.
+   *
+   * \param call The call to be represented.
+   * \return A JSON representation of a specific operator.
+   */
+  std::shared_ptr<JSONGraphNode> CreateOp(const CallNode* cn);
+  std::shared_ptr<JSONGraphNode> CreateCompositeConvolution(const CallNode* cn);
+
+  /* \brief Transposed constant tensors to serialize. Arm Compute Library expects constant tensors
+   * in OHWI format. */
+  Array<runtime::NDArray> constants_;
+};
+
+/*!
+ * \brief Pre-process a module containing functions ready for ACL codegen.
+ *
+ * For now we enforce OHWI kernel layout and fold the transforms away.
+ *
+ * \param mod The module to be pre-processed.
+ * \return The processed module.
+ */
+IRModule PreProcessModule(const IRModule& mod);
+
+/*!
+ * \brief Create a runtime module for ACL.
+ *
+ * This consists of a series of "serialized functions" which each represent a
+ * sub-graph to be computed by ACL and will each be executed independently from
+ * one another. Each function consists of serialized JSON describing the sub-graph
+ * and serialized constant tensors.
+ *
+ * \note The ACL runtime module only currently supports a single operator per
+ * sub-graph currently.
+ *
+ * \param ref The ext_func Relay expression/module to be executed using extern ops.
+ * \return A runtime module.
+ */
+runtime::Module ACLCompiler(const ObjectRef& ref);
+
+/*!
+ * \brief Get the external symbol of the Relay function name.
+ *
+ * \param func The provided function.
+ *
+ * \return An external symbol.
+ */
+std::string GetExtSymbol(const Function& func) {
+  const auto name_node = func->GetAttr<String>(tvm::attr::kGlobalSymbol);
+  CHECK(name_node.defined()) << "Fail to retrieve external symbol.";
+  return std::string(name_node.value());
+}
+
+TVM_REGISTER_GLOBAL("relay.ext.arm_compute_lib").set_body_typed(ACLCompiler);
+
+/*!
+ * \brief Check whether ACL graph runtime is used.
+ * \return True if ACL graph runtime is enabled, False if not.
+ */
+inline constexpr bool IsACLRuntimeEnabled() {
+#if TVM_GRAPH_RUNTIME_ACL

Review comment:
       should it be `TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB` as this is what you added in the cmake




----------------------------------------------------------------
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 #5915: [BYOC][Contrib] Arm Compute Library integration

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



##########
File path: src/runtime/contrib/arm_compute_lib/acl_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.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif

Review comment:
       The utils functions and allocator are wrapped in another namespace `arm_compute_lib`, using this namespace here avoids writing `arm_compute_lib` when we want to use these functions. Probably best to remove the namespace altogether.




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