You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2022/11/30 14:34:26 UTC

[GitHub] [tvm] ashutosh-arm opened a new pull request, #13522: [CodegenC] Explicit forward function declarations

ashutosh-arm opened a new pull request, #13522:
URL: https://github.com/apache/tvm/pull/13522

   armclang 6.19 does not support implicit function declarations. This commit adds support for generating forward function declarations in the C file generated for __tvm_main__. All the non-pure extern functions called from __tvm_main__ will be declared explicitly in this file.
   
   cc @Mousius @lhutton1 


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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1040726597


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):

Review Comment:
   If we can leave it here for now, it will be easier to find it by name when its cleaned up. A similar function is being used by a lot of other tests. I plan to clean it up soon for Arm targets :smile: 



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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
src/target/source/codegen_c_host.cc:
##########
@@ -93,18 +94,46 @@ void CodeGenCHost::AddFunction(const PrimFunc& f) {
   }
 }
 
-void CodeGenCHost::PrintFuncPrefix() {  // NOLINT(*)
-  stream << "#ifdef __cplusplus\n"
-         << "extern \"C\"\n"
-         << "#endif\n"
-         << "TVM_DLL int32_t";
+void CodeGenCHost::GenerateForwardFunctionDeclarations(String global_symbol,
+                                                       const Array<PrimExpr>& args) {
+  for (auto& func_already_defined : GetFunctionNames()) {
+    if (global_symbol == func_already_defined) {
+      return;
+    }
+  }
+  this->PrintFuncPrefix(fwd_decl_stream);
+  fwd_decl_stream << " " << global_symbol << "(";
+  for (size_t i = 1; i < args.size(); ++i) {

Review Comment:
   Wouldn't it be better to share the same logic though? Then we can guarantee the forward declarations will always match the function prints later? Otherwise we'd have to update in both places should we change how we print the function definition.



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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
src/target/source/codegen_c_host.cc:
##########
@@ -93,18 +94,46 @@ void CodeGenCHost::AddFunction(const PrimFunc& f) {
   }
 }
 
-void CodeGenCHost::PrintFuncPrefix() {  // NOLINT(*)
-  stream << "#ifdef __cplusplus\n"
-         << "extern \"C\"\n"
-         << "#endif\n"
-         << "TVM_DLL int32_t";
+void CodeGenCHost::GenerateForwardFunctionDeclarations(String global_symbol,
+                                                       const Array<PrimExpr>& args) {
+  for (auto& func_already_defined : GetFunctionNames()) {
+    if (global_symbol == func_already_defined) {
+      return;
+    }
+  }
+  this->PrintFuncPrefix(fwd_decl_stream);
+  fwd_decl_stream << " " << global_symbol << "(";
+  for (size_t i = 1; i < args.size(); ++i) {

Review Comment:
   Resolved offline, this requires more information to be threaded and is out of scope here, can refactor later if necessary



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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1037073763


##########
src/target/source/codegen_c_host.cc:
##########
@@ -93,18 +94,46 @@ void CodeGenCHost::AddFunction(const PrimFunc& f) {
   }
 }
 
-void CodeGenCHost::PrintFuncPrefix() {  // NOLINT(*)
-  stream << "#ifdef __cplusplus\n"
-         << "extern \"C\"\n"
-         << "#endif\n"
-         << "TVM_DLL int32_t";
+void CodeGenCHost::GenerateForwardFunctionDeclarations(String global_symbol,
+                                                       const Array<PrimExpr>& args) {
+  for (auto& func_already_defined : GetFunctionNames()) {
+    if (global_symbol == func_already_defined) {
+      return;
+    }
+  }
+  this->PrintFuncPrefix(fwd_decl_stream);
+  fwd_decl_stream << " " << global_symbol << "(";
+  for (size_t i = 1; i < args.size(); ++i) {

Review Comment:
   I had a look here. It seems this handles AllocateNodes scopes and aliasing that are both optional in forward function declaration. So, I think we can skip the additional handling around types.



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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1037188599


##########
tests/python/contrib/test_cmsisnn/test_forward_function_declarations.py:
##########
@@ -0,0 +1,146 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""CMSIS-NN: test forward function declarations"""
+
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, run_and_check, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+)
+from .utils import skip_if_no_reference_system, get_range_for_dtype_str
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(
+    tflite_model_buf,
+    input_data,
+    input_node,
+):
+    """Converts TFLite model to Relay module and params"""
+
+    def convert_to_list(x):
+        if not isinstance(x, list):
+            x = [x]
+        return x
+
+    # TFLite.Model.Model has changed to TFLite.Model from 1.14 to 2.1
+    try:
+        import tflite.Model
+
+        tflite_model = tflite.Model.Model.GetRootAsModel(tflite_model_buf, 0)
+    except AttributeError:
+        import tflite
+
+        tflite_model = tflite.Model.GetRootAsModel(tflite_model_buf, 0)
+    except ImportError:
+        raise ImportError("The tflite package must be installed")
+
+    input_data = convert_to_list(input_data)
+    input_node = convert_to_list(input_node)
+
+    shape_dict = {}
+    dtype_dict = {}
+    for i, name in enumerate(input_node):
+        shape_dict[name] = input_data[i].shape
+        dtype_dict[name] = input_data[i].dtype.name
+
+    mod, params = relay.frontend.from_tflite(
+        tflite_model, shape_dict=shape_dict, dtype_dict=dtype_dict
+    )
+
+    return mod, params
+
+
+@skip_if_no_reference_system
+@tvm.testing.requires_package("tflite")
+@tvm.testing.requires_cmsisnn
+@pytest.mark.parametrize("test_runner", [AOT_CORSTONE300_RUNNER, AOT_USMP_CORSTONE300_RUNNER])
+def test_cnn_small(test_runner):
+    """Download a small network and tests TVM via CMSIS-NN output against TFLite output"""
+    # download the model
+    base_url = (
+        "https://github.com/ARM-software/ML-zoo/raw/"
+        "48a22ee22325d15d2371a6df24eb7d67e21dcc97"
+        "/models/keyword_spotting/cnn_small/tflite_int8"
+    )
+    file_to_download = "cnn_s_quantized.tflite"
+    file_saved = "cnn_s_quantized_15Dec2021.tflite"
+    model_file = download_testdata("{}/{}".format(base_url, file_to_download), file_saved)
+
+    with open(model_file, "rb") as f:
+        tflite_model_buf = f.read()
+
+    input_shape = (1, 490)
+    dtype = "int8"
+    in_min, in_max = get_range_for_dtype_str(dtype)
+    rng = np.random.default_rng(12345)
+    input_data = rng.integers(in_min, high=in_max, size=input_shape, dtype=dtype)
+
+    orig_mod, params = _convert_to_relay(tflite_model_buf, input_data, "input")
+    cmsisnn_mod = cmsisnn.partition_for_cmsisnn(orig_mod, params)
+
+    # validate CMSIS-NN output against CPU output
+    interface_api = "c"
+    use_unpacked_api = True
+    inputs = {"input": input_data}
+    params = {}
+    output_list = generate_ref_data(orig_mod["main"], inputs, params)
+    compiled_models = compile_models(
+        AOTTestModel(
+            module=cmsisnn_mod,
+            inputs=inputs,
+            outputs=output_list,
+            params=None,
+            output_tolerance=1,
+        ),
+        interface_api,
+        use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    # Forward function declaration increases the number of times a function name appears under
+    # __tvm__main. Validate this frequency for native, offloaded and allocation functions.

Review Comment:
   After the offline discussion about it, it was decided to produce the forward declarations for `__tvm_main__` only. Other CodegenC instances could be using functions from external libs for which the header might be included by the codegen and forward declarations may not be needed in such cases.



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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
tests/python/contrib/test_cmsisnn/test_forward_function_declarations.py:
##########
@@ -0,0 +1,146 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""CMSIS-NN: test forward function declarations"""
+
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, run_and_check, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+)
+from .utils import skip_if_no_reference_system, get_range_for_dtype_str
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(
+    tflite_model_buf,
+    input_data,
+    input_node,
+):
+    """Converts TFLite model to Relay module and params"""
+
+    def convert_to_list(x):
+        if not isinstance(x, list):
+            x = [x]
+        return x
+
+    # TFLite.Model.Model has changed to TFLite.Model from 1.14 to 2.1
+    try:
+        import tflite.Model
+
+        tflite_model = tflite.Model.Model.GetRootAsModel(tflite_model_buf, 0)
+    except AttributeError:
+        import tflite
+
+        tflite_model = tflite.Model.GetRootAsModel(tflite_model_buf, 0)
+    except ImportError:
+        raise ImportError("The tflite package must be installed")
+
+    input_data = convert_to_list(input_data)
+    input_node = convert_to_list(input_node)
+
+    shape_dict = {}
+    dtype_dict = {}
+    for i, name in enumerate(input_node):
+        shape_dict[name] = input_data[i].shape
+        dtype_dict[name] = input_data[i].dtype.name
+
+    mod, params = relay.frontend.from_tflite(
+        tflite_model, shape_dict=shape_dict, dtype_dict=dtype_dict
+    )
+
+    return mod, params
+
+
+@skip_if_no_reference_system
+@tvm.testing.requires_package("tflite")
+@tvm.testing.requires_cmsisnn
+@pytest.mark.parametrize("test_runner", [AOT_CORSTONE300_RUNNER, AOT_USMP_CORSTONE300_RUNNER])
+def test_cnn_small(test_runner):
+    """Download a small network and tests TVM via CMSIS-NN output against TFLite output"""
+    # download the model
+    base_url = (
+        "https://github.com/ARM-software/ML-zoo/raw/"
+        "48a22ee22325d15d2371a6df24eb7d67e21dcc97"
+        "/models/keyword_spotting/cnn_small/tflite_int8"
+    )
+    file_to_download = "cnn_s_quantized.tflite"
+    file_saved = "cnn_s_quantized_15Dec2021.tflite"
+    model_file = download_testdata("{}/{}".format(base_url, file_to_download), file_saved)
+
+    with open(model_file, "rb") as f:
+        tflite_model_buf = f.read()
+
+    input_shape = (1, 490)
+    dtype = "int8"
+    in_min, in_max = get_range_for_dtype_str(dtype)
+    rng = np.random.default_rng(12345)
+    input_data = rng.integers(in_min, high=in_max, size=input_shape, dtype=dtype)
+
+    orig_mod, params = _convert_to_relay(tflite_model_buf, input_data, "input")
+    cmsisnn_mod = cmsisnn.partition_for_cmsisnn(orig_mod, params)
+
+    # validate CMSIS-NN output against CPU output
+    interface_api = "c"
+    use_unpacked_api = True
+    inputs = {"input": input_data}
+    params = {}
+    output_list = generate_ref_data(orig_mod["main"], inputs, params)
+    compiled_models = compile_models(
+        AOTTestModel(
+            module=cmsisnn_mod,
+            inputs=inputs,
+            outputs=output_list,
+            params=None,
+            output_tolerance=1,
+        ),
+        interface_api,
+        use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    # Forward function declaration increases the number of times a function name appears under
+    # __tvm__main. Validate this frequency for native, offloaded and allocation functions.

Review Comment:
   Isn't it under the host codegen'd file not under the `__tvm__main` function?



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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1040726910


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)

Review Comment:
   ACK



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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1037186966


##########
tests/python/contrib/test_cmsisnn/test_forward_function_declarations.py:
##########
@@ -0,0 +1,146 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""CMSIS-NN: test forward function declarations"""

Review Comment:
   Yep, that makes sense. 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.

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1040773516


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):
+    """
+    Produces the min,max for a give data type.
+
+    Parameters
+    ----------
+    dtype : str
+        a type string (e.g., int8)
+
+    Returns
+    -------
+    type_info.min : int
+        the minimum of the range
+    type_info.max : int
+        the maximum of the range
+    """
+
+    try:
+        type_info = np.iinfo(dtype)
+    except ValueError:
+        type_info = np.finfo(dtype)
+    return type_info.min, type_info.max
+
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(
+    tflite_model_buf,
+    input_data,
+    input_node,
+):
+    """Converts TFLite model to Relay module and params"""
+
+    def convert_to_list(x):
+        if not isinstance(x, list):
+            x = [x]
+        return x
+
+    # TFLite.Model.Model has changed to TFLite.Model from 1.14 to 2.1
+    try:
+        import tflite.Model
+
+        tflite_model = tflite.Model.Model.GetRootAsModel(tflite_model_buf, 0)
+    except AttributeError:
+        import tflite
+
+        tflite_model = tflite.Model.GetRootAsModel(tflite_model_buf, 0)
+    except ImportError:
+        raise ImportError("The tflite package must be installed")
+
+    input_data = convert_to_list(input_data)
+    input_node = convert_to_list(input_node)
+
+    shape_dict = {}
+    dtype_dict = {}
+    for i, name in enumerate(input_node):
+        shape_dict[name] = input_data[i].shape
+        dtype_dict[name] = input_data[i].dtype.name
+
+    mod, params = relay.frontend.from_tflite(
+        tflite_model, shape_dict=shape_dict, dtype_dict=dtype_dict
+    )
+
+    return mod, params
+
+
+def _change_ndarray_layout(arr, src_layout, dst_layout):
+    """Makes a copy of an ndarray, reshaping it to a new data layout.
+
+    Parameter
+    ---------
+    arr : numpy.ndarray
+        The ndarray to be reformatted.
+
+    src_layout : str
+        The current layout of the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    dst_layout : str
+        The desired layout of new the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    Returns
+    -------
+    dst_shape : numpy.ndarray
+        A copy of the ndarray with the new layout.
+    """
+    assert src_layout.isalpha() and dst_layout.isalpha()
+    axis_order = [src_layout.index(c) for c in dst_layout]
+    return np.transpose(arr, axis_order)
+
+
+@tvm.testing.requires_package("tflite")
+@tvm.testing.requires_cmsisnn
+@pytest.mark.parametrize("test_runner", [AOT_CORSTONE300_RUNNER, AOT_USMP_CORSTONE300_RUNNER])
+def test_external_calls(test_runner):
+    """Download a small network and partition for CMSIS-NN to test forward declarations for external
+    calls outside of __tvm_main__."""
+    # download the model
+    base_url = (
+        "https://github.com/ARM-software/ML-zoo/raw/"
+        "48a22ee22325d15d2371a6df24eb7d67e21dcc97"
+        "/models/keyword_spotting/cnn_small/tflite_int8"
+    )
+    file_to_download = "cnn_s_quantized.tflite"
+    file_saved = "cnn_s_quantized_15Dec2021.tflite"
+    model_file = download_testdata("{}/{}".format(base_url, file_to_download), file_saved)
+
+    with open(model_file, "rb") as f:
+        tflite_model_buf = f.read()
+
+    input_shape = (1, 490)
+    dtype = "int8"
+    in_min, in_max = get_range_for_dtype_str(dtype)
+    rng = np.random.default_rng(12345)
+    input_data = rng.integers(in_min, high=in_max, size=input_shape, dtype=dtype)
+
+    orig_mod, params = _convert_to_relay(tflite_model_buf, input_data, "input")
+    cmsisnn_mod = cmsisnn.partition_for_cmsisnn(orig_mod, params)
+
+    # validate CMSIS-NN output against CPU output
+    interface_api = "c"
+    use_unpacked_api = True
+    inputs = {"input": input_data}
+    params = {}
+    output_list = generate_ref_data(orig_mod["main"], inputs, params)
+    compiled_models = compile_models(
+        AOTTestModel(
+            module=cmsisnn_mod,
+            inputs=inputs,
+            outputs=output_list,
+            params=None,
+            output_tolerance=1,
+        ),
+        interface_api,
+        use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    # Forward function declaration increases the number of times a function name appears under
+    # __tvm__main. Validate this frequency for native, offloaded and allocation functions.

Review Comment:
   I have rephrased it now. The language was confusing previously.



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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
tests/python/contrib/test_cmsisnn/test_forward_function_declarations.py:
##########
@@ -0,0 +1,146 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""CMSIS-NN: test forward function declarations"""

Review Comment:
   This seems broader than just CMSIS-NN, would it be better to add these tests to something like an AOT test case?



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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1040981648


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):
+    """
+    Produces the min,max for a give data type.
+
+    Parameters
+    ----------
+    dtype : str
+        a type string (e.g., int8)
+
+    Returns
+    -------
+    type_info.min : int
+        the minimum of the range
+    type_info.max : int
+        the maximum of the range
+    """
+
+    try:
+        type_info = np.iinfo(dtype)
+    except ValueError:
+        type_info = np.finfo(dtype)
+    return type_info.min, type_info.max
+
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(

Review Comment:
   I have extended that class to load from file. Thanks for the suggestion :partying_face: 



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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):

Review Comment:
   Ok dokie 😸 



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

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

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


[GitHub] [tvm] Mousius merged pull request #13522: [CodegenC] Explicit forward function declarations

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


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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
src/target/source/codegen_c_host.cc:
##########
@@ -93,18 +94,46 @@ void CodeGenCHost::AddFunction(const PrimFunc& f) {
   }
 }
 
-void CodeGenCHost::PrintFuncPrefix() {  // NOLINT(*)
-  stream << "#ifdef __cplusplus\n"
-         << "extern \"C\"\n"
-         << "#endif\n"
-         << "TVM_DLL int32_t";
+void CodeGenCHost::GenerateForwardFunctionDeclarations(String global_symbol,
+                                                       const Array<PrimExpr>& args) {
+  for (auto& func_already_defined : GetFunctionNames()) {
+    if (global_symbol == func_already_defined) {
+      return;
+    }
+  }
+  this->PrintFuncPrefix(fwd_decl_stream);
+  fwd_decl_stream << " " << global_symbol << "(";
+  for (size_t i = 1; i < args.size(); ++i) {

Review Comment:
   Would it make sense to have a `PrintFunctionPrototype`? 
   
   It seems there's more logic here: https://github.com/apache/tvm/blob/a70c0649a895cf600e600bb3a1f49f6269aa71db/src/target/source/codegen_c.cc#L92-L119 which could apply?



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

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

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


[GitHub] [tvm] tvm-bot commented on pull request #13522: [CodegenC] Explicit forward function declarations

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

   <!---bot-comment-->
   
   Thanks for contributing to TVM! Please refer to the contributing guidelines https://tvm.apache.org/docs/contribute/ for useful information and tips. Please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @-ing them in a comment.
   
   <!--bot-comment-ccs-start-->
    * No users to tag found in teams: `codegenc` <sub>See [#10317](https://github.com/apache/tvm/issues/10317) for details</sub><!--bot-comment-ccs-end-->
   
   <sub>Generated by [tvm-bot](https://github.com/apache/tvm/blob/main/ci/README.md#github-actions)</sub>


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

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

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


[GitHub] [tvm] Mousius commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

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


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)

Review Comment:
   Can we use `@tvm.testing.requires_corstone300` instead?



##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):

Review Comment:
   This function is only used once and with a fixed dtype so it doesn't feel necessary to include it just for that, could we either move it to a testing util or just hardcode the range? 



##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):
+    """
+    Produces the min,max for a give data type.
+
+    Parameters
+    ----------
+    dtype : str
+        a type string (e.g., int8)
+
+    Returns
+    -------
+    type_info.min : int
+        the minimum of the range
+    type_info.max : int
+        the maximum of the range
+    """
+
+    try:
+        type_info = np.iinfo(dtype)
+    except ValueError:
+        type_info = np.finfo(dtype)
+    return type_info.min, type_info.max
+
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(

Review Comment:
   Can we add a `load_from_file` to here:
    https://github.com/apache/tvm/blob/72e11baabb0e3a7e311c4b3490b729641c489555/python/tvm/relay/testing/tflite.py#L29
   
   So it's just:
   ```
   TFLiteModel.from_buffer(buf).convert_to_relay()
   ```
   rather than recreating parts of that class here?



##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):
+    """
+    Produces the min,max for a give data type.
+
+    Parameters
+    ----------
+    dtype : str
+        a type string (e.g., int8)
+
+    Returns
+    -------
+    type_info.min : int
+        the minimum of the range
+    type_info.max : int
+        the maximum of the range
+    """
+
+    try:
+        type_info = np.iinfo(dtype)
+    except ValueError:
+        type_info = np.finfo(dtype)
+    return type_info.min, type_info.max
+
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(
+    tflite_model_buf,
+    input_data,
+    input_node,
+):
+    """Converts TFLite model to Relay module and params"""
+
+    def convert_to_list(x):
+        if not isinstance(x, list):
+            x = [x]
+        return x
+
+    # TFLite.Model.Model has changed to TFLite.Model from 1.14 to 2.1
+    try:
+        import tflite.Model
+
+        tflite_model = tflite.Model.Model.GetRootAsModel(tflite_model_buf, 0)
+    except AttributeError:
+        import tflite
+
+        tflite_model = tflite.Model.GetRootAsModel(tflite_model_buf, 0)
+    except ImportError:
+        raise ImportError("The tflite package must be installed")
+
+    input_data = convert_to_list(input_data)
+    input_node = convert_to_list(input_node)
+
+    shape_dict = {}
+    dtype_dict = {}
+    for i, name in enumerate(input_node):
+        shape_dict[name] = input_data[i].shape
+        dtype_dict[name] = input_data[i].dtype.name
+
+    mod, params = relay.frontend.from_tflite(
+        tflite_model, shape_dict=shape_dict, dtype_dict=dtype_dict
+    )
+
+    return mod, params
+
+
+def _change_ndarray_layout(arr, src_layout, dst_layout):
+    """Makes a copy of an ndarray, reshaping it to a new data layout.
+
+    Parameter
+    ---------
+    arr : numpy.ndarray
+        The ndarray to be reformatted.
+
+    src_layout : str
+        The current layout of the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    dst_layout : str
+        The desired layout of new the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    Returns
+    -------
+    dst_shape : numpy.ndarray
+        A copy of the ndarray with the new layout.
+    """
+    assert src_layout.isalpha() and dst_layout.isalpha()
+    axis_order = [src_layout.index(c) for c in dst_layout]
+    return np.transpose(arr, axis_order)
+
+
+@tvm.testing.requires_package("tflite")
+@tvm.testing.requires_cmsisnn
+@pytest.mark.parametrize("test_runner", [AOT_CORSTONE300_RUNNER, AOT_USMP_CORSTONE300_RUNNER])
+def test_external_calls(test_runner):
+    """Download a small network and partition for CMSIS-NN to test forward declarations for external
+    calls outside of __tvm_main__."""
+    # download the model
+    base_url = (
+        "https://github.com/ARM-software/ML-zoo/raw/"
+        "48a22ee22325d15d2371a6df24eb7d67e21dcc97"
+        "/models/keyword_spotting/cnn_small/tflite_int8"
+    )
+    file_to_download = "cnn_s_quantized.tflite"
+    file_saved = "cnn_s_quantized_15Dec2021.tflite"
+    model_file = download_testdata("{}/{}".format(base_url, file_to_download), file_saved)
+
+    with open(model_file, "rb") as f:
+        tflite_model_buf = f.read()
+
+    input_shape = (1, 490)
+    dtype = "int8"
+    in_min, in_max = get_range_for_dtype_str(dtype)
+    rng = np.random.default_rng(12345)
+    input_data = rng.integers(in_min, high=in_max, size=input_shape, dtype=dtype)
+
+    orig_mod, params = _convert_to_relay(tflite_model_buf, input_data, "input")
+    cmsisnn_mod = cmsisnn.partition_for_cmsisnn(orig_mod, params)
+
+    # validate CMSIS-NN output against CPU output
+    interface_api = "c"
+    use_unpacked_api = True
+    inputs = {"input": input_data}
+    params = {}
+    output_list = generate_ref_data(orig_mod["main"], inputs, params)
+    compiled_models = compile_models(
+        AOTTestModel(
+            module=cmsisnn_mod,
+            inputs=inputs,
+            outputs=output_list,
+            params=None,
+            output_tolerance=1,
+        ),
+        interface_api,
+        use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    # Forward function declaration increases the number of times a function name appears under
+    # __tvm__main. Validate this frequency for native, offloaded and allocation functions.
+    lib_mod = compiled_models[0].executor_factory.lib.imported_modules[0]
+    main_source = lib_mod.get_source()
+    assert (
+        main_source.count("TVMBackendAllocWorkspace") == 3
+        or main_source.count("TVMBackendAllocWorkspace") == 0
+    )
+    assert main_source.count("tvmgen_default_fused_reshape") == 2
+    assert main_source.count("tvmgen_default_cmsis_nn_main") == 12
+    cmsisnn_source = lib_mod.imported_modules[0].get_source()
+    assert cmsisnn_source.count("arm_convolve_wrapper") == 1
+    assert cmsisnn_source.count("arm_fully_connected") == 3
+    assert cmsisnn_source.count("arm_softmax") == 1
+
+
+@parametrize_aot_options
+def test_internal_calls(interface_api, use_unpacked_api, test_runner):
+    """Test for all internal function calls. No forward declarations are expected here."""
+    dtype = "float32"
+    groups = 32
+    weight_shape = 1
+    ishape = (1, 32, 14, 14)
+    wshape = (32, weight_shape, 3, 3)
+    pass_config = {"tir.usmp.enable": True}
+    test_runner = AOTTestRunner(
+        makefile=test_runner.makefile,
+        prologue=test_runner.prologue,
+        epilogue=test_runner.epilogue,
+        includes=test_runner.includes,
+        parameters=test_runner.parameters,
+        pass_config=pass_config,
+    )
+
+    data0 = relay.var("data", shape=ishape, dtype=dtype)
+    weight0 = relay.var("weight", shape=wshape, dtype=dtype)
+    out = relay.nn.conv2d(data0, weight0, kernel_size=(3, 3), padding=(1, 1), groups=groups)
+    main_f = relay.Function([data0, weight0], out)
+    mod = tvm.IRModule()
+    mod["main"] = main_f
+    mod = tvm.relay.transform.InferType()(mod)
+
+    i_data = np.random.uniform(0, 1, ishape).astype(dtype)
+    w1_data = np.random.uniform(0, 1, wshape).astype(dtype)
+
+    inputs = OrderedDict([("data", i_data), ("weight", w1_data)])
+
+    output_list = generate_ref_data(mod, inputs)
+    compiled_models = compile_models(
+        models=AOTTestModel(module=mod, inputs=inputs, outputs=output_list),
+        interface_api=interface_api,
+        use_unpacked_api=use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    lib_mod = compiled_models[0].executor_factory.lib.imported_modules[0]
+    main_source = lib_mod.get_source()
+    assert main_source.count("tvmgen_default_fused_nn_contrib_depthwise_conv2d_NCHWc") == 2
+    assert main_source.count("tvmgen_default_fused_layout_transform") == 6
+
+
+@skip_if_no_reference_system
+@tvm.testing.requires_cmsisnn
+def test_tensorized_calls():
+    """Test a subgraph with a mix of internal and tensorized calls."""
+    data_shape, kernel_size, num_filter, groups, strides, padding, dilation = (
+        (1, 32, 32, 16),
+        (3, 3),
+        16,
+        1,
+        1,
+        (0, 2, 2, 0),
+        1,
+    )
+    in_dtype = "int8"
+    data_layout = "NHWC"
+    kernel_layout = "HWOI"
+    ref_kernel_layout = "HWIO"
+    out_layout = "NHWC"
+    schedule_name = "conv2d_nhwc_dsp.arm_cpu"
+
+    ref_input_data = np.random.randint(low=-128, high=127, size=data_shape, dtype=in_dtype)
+    ref_input_var = relay.var("input", relay.TensorType(data_shape, in_dtype))  # NHWC layout
+    kernel_shape = (*kernel_size, data_shape[-1] // groups, num_filter)  # HWIO layout
+    ref_kernel_data = np.random.randint(low=-10, high=10, size=kernel_shape, dtype=in_dtype)
+
+    ref_relay_op = relay.op.nn.conv2d(
+        ref_input_var,
+        relay.const(_change_ndarray_layout(ref_kernel_data, "HWIO", ref_kernel_layout)),
+        kernel_size=kernel_size,
+        strides=strides,
+        padding=padding,
+        groups=groups,
+        dilation=(dilation, dilation),
+        data_layout="NHWC",
+        kernel_layout=ref_kernel_layout,
+        out_dtype="int32",
+        out_layout="NHWC",
+    )
+    ref_module = tvm.IRModule.from_expr(relay.Function([ref_input_var], ref_relay_op))
+    ref_outputs = generate_ref_data(ref_module, {"input": ref_input_data})
+
+    # Reshape output dictionary to match out_layout
+    assert len(ref_outputs) == 1
+    output_tensor_name, output_tensor = next(iter(ref_outputs.items()))
+    ref_outputs[output_tensor_name] = _change_ndarray_layout(output_tensor, "NHWC", out_layout)
+
+    test_input_data = _change_ndarray_layout(ref_input_data, "NHWC", data_layout)
+    test_input_var = relay.var("input", relay.TensorType(test_input_data.shape, in_dtype))
+    test_kernel_data = _change_ndarray_layout(ref_kernel_data, "HWIO", kernel_layout)
+
+    test_relay_op = relay.op.nn.conv2d(
+        test_input_var,
+        relay.const(test_kernel_data),
+        kernel_size=kernel_size,
+        strides=strides,
+        padding=padding,
+        groups=groups,
+        dilation=(dilation, dilation),
+        data_layout=data_layout,
+        kernel_layout=kernel_layout,
+        out_dtype="int32",
+        out_layout=out_layout,
+    )
+    test_function = relay.Function([test_input_var], test_relay_op)
+    test_model = AOTTestModel(
+        module=tvm.IRModule.from_expr(test_function),
+        inputs={"input": test_input_data},
+        outputs=ref_outputs,
+    )
+    compiled_models = compile_models(
+        test_model,
+        interface_api="c",
+        use_unpacked_api=True,
+        pass_config=AOT_CORSTONE300_RUNNER.pass_config,
+        target=f"c -keys=arm_cpu -mcpu=cortex-m7",
+        schedule_name=schedule_name,
+    )
+
+    lib_mod = compiled_models[0].executor_factory.lib.imported_modules[0]
+    main_source = lib_mod.get_source()
+    assert main_source.count("tvmgen_default_fused_nn_conv2d") == 2
+    assert main_source.count("gemm_") == 13

Review Comment:
   We should consider using a test-oriented schedules/BYOC here rather than these tests being dependent on a specific implementation elsewhere in the codebase - this is fine for now though.



##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):
+    """
+    Produces the min,max for a give data type.
+
+    Parameters
+    ----------
+    dtype : str
+        a type string (e.g., int8)
+
+    Returns
+    -------
+    type_info.min : int
+        the minimum of the range
+    type_info.max : int
+        the maximum of the range
+    """
+
+    try:
+        type_info = np.iinfo(dtype)
+    except ValueError:
+        type_info = np.finfo(dtype)
+    return type_info.min, type_info.max
+
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(
+    tflite_model_buf,
+    input_data,
+    input_node,
+):
+    """Converts TFLite model to Relay module and params"""
+
+    def convert_to_list(x):
+        if not isinstance(x, list):
+            x = [x]
+        return x
+
+    # TFLite.Model.Model has changed to TFLite.Model from 1.14 to 2.1
+    try:
+        import tflite.Model
+
+        tflite_model = tflite.Model.Model.GetRootAsModel(tflite_model_buf, 0)
+    except AttributeError:
+        import tflite
+
+        tflite_model = tflite.Model.GetRootAsModel(tflite_model_buf, 0)
+    except ImportError:
+        raise ImportError("The tflite package must be installed")
+
+    input_data = convert_to_list(input_data)
+    input_node = convert_to_list(input_node)
+
+    shape_dict = {}
+    dtype_dict = {}
+    for i, name in enumerate(input_node):
+        shape_dict[name] = input_data[i].shape
+        dtype_dict[name] = input_data[i].dtype.name
+
+    mod, params = relay.frontend.from_tflite(
+        tflite_model, shape_dict=shape_dict, dtype_dict=dtype_dict
+    )
+
+    return mod, params
+
+
+def _change_ndarray_layout(arr, src_layout, dst_layout):
+    """Makes a copy of an ndarray, reshaping it to a new data layout.
+
+    Parameter
+    ---------
+    arr : numpy.ndarray
+        The ndarray to be reformatted.
+
+    src_layout : str
+        The current layout of the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    dst_layout : str
+        The desired layout of new the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    Returns
+    -------
+    dst_shape : numpy.ndarray
+        A copy of the ndarray with the new layout.
+    """
+    assert src_layout.isalpha() and dst_layout.isalpha()
+    axis_order = [src_layout.index(c) for c in dst_layout]
+    return np.transpose(arr, axis_order)
+
+
+@tvm.testing.requires_package("tflite")
+@tvm.testing.requires_cmsisnn
+@pytest.mark.parametrize("test_runner", [AOT_CORSTONE300_RUNNER, AOT_USMP_CORSTONE300_RUNNER])
+def test_external_calls(test_runner):
+    """Download a small network and partition for CMSIS-NN to test forward declarations for external
+    calls outside of __tvm_main__."""
+    # download the model
+    base_url = (
+        "https://github.com/ARM-software/ML-zoo/raw/"
+        "48a22ee22325d15d2371a6df24eb7d67e21dcc97"
+        "/models/keyword_spotting/cnn_small/tflite_int8"
+    )
+    file_to_download = "cnn_s_quantized.tflite"
+    file_saved = "cnn_s_quantized_15Dec2021.tflite"
+    model_file = download_testdata("{}/{}".format(base_url, file_to_download), file_saved)
+
+    with open(model_file, "rb") as f:
+        tflite_model_buf = f.read()
+
+    input_shape = (1, 490)
+    dtype = "int8"
+    in_min, in_max = get_range_for_dtype_str(dtype)
+    rng = np.random.default_rng(12345)
+    input_data = rng.integers(in_min, high=in_max, size=input_shape, dtype=dtype)
+
+    orig_mod, params = _convert_to_relay(tflite_model_buf, input_data, "input")
+    cmsisnn_mod = cmsisnn.partition_for_cmsisnn(orig_mod, params)
+
+    # validate CMSIS-NN output against CPU output
+    interface_api = "c"
+    use_unpacked_api = True
+    inputs = {"input": input_data}
+    params = {}
+    output_list = generate_ref_data(orig_mod["main"], inputs, params)
+    compiled_models = compile_models(
+        AOTTestModel(
+            module=cmsisnn_mod,
+            inputs=inputs,
+            outputs=output_list,
+            params=None,
+            output_tolerance=1,
+        ),
+        interface_api,
+        use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    # Forward function declaration increases the number of times a function name appears under
+    # __tvm__main. Validate this frequency for native, offloaded and allocation functions.

Review Comment:
   This doesn't produce forward declarations inside `__tvm_main__`, it produces them inside the host code generation for functions that are called within `__tvm_main__`?



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

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

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


[GitHub] [tvm] ashutosh-arm commented on a diff in pull request #13522: [CodegenC] Explicit forward function declarations

Posted by GitBox <gi...@apache.org>.
ashutosh-arm commented on code in PR #13522:
URL: https://github.com/apache/tvm/pull/13522#discussion_r1040776186


##########
tests/python/relay/aot/test_crt_forward_declarations.py:
##########
@@ -0,0 +1,325 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+"""test forward function declarations codegen by CodegenCHost."""
+
+from collections import OrderedDict
+import pytest
+import numpy as np
+
+import tvm.testing
+from tvm import relay
+from tvm.contrib.download import download_testdata
+from tvm.relay.op.contrib import cmsisnn
+from tvm.testing.aot import AOTTestModel, compile_models, generate_ref_data
+from tvm.micro.testing.aot_test_utils import (
+    AOT_CORSTONE300_RUNNER,
+    AOT_USMP_CORSTONE300_RUNNER,
+    parametrize_aot_options,
+    AOTTestRunner,
+)
+
+
+def skip_if_no_reference_system(func):
+    return tvm.testing.skip_if_32bit(reason="Reference system unavailable in i386 container")(func)
+
+
+def get_range_for_dtype_str(dtype):
+    """
+    Produces the min,max for a give data type.
+
+    Parameters
+    ----------
+    dtype : str
+        a type string (e.g., int8)
+
+    Returns
+    -------
+    type_info.min : int
+        the minimum of the range
+    type_info.max : int
+        the maximum of the range
+    """
+
+    try:
+        type_info = np.iinfo(dtype)
+    except ValueError:
+        type_info = np.finfo(dtype)
+    return type_info.min, type_info.max
+
+
+# pylint: disable=import-outside-toplevel
+def _convert_to_relay(
+    tflite_model_buf,
+    input_data,
+    input_node,
+):
+    """Converts TFLite model to Relay module and params"""
+
+    def convert_to_list(x):
+        if not isinstance(x, list):
+            x = [x]
+        return x
+
+    # TFLite.Model.Model has changed to TFLite.Model from 1.14 to 2.1
+    try:
+        import tflite.Model
+
+        tflite_model = tflite.Model.Model.GetRootAsModel(tflite_model_buf, 0)
+    except AttributeError:
+        import tflite
+
+        tflite_model = tflite.Model.GetRootAsModel(tflite_model_buf, 0)
+    except ImportError:
+        raise ImportError("The tflite package must be installed")
+
+    input_data = convert_to_list(input_data)
+    input_node = convert_to_list(input_node)
+
+    shape_dict = {}
+    dtype_dict = {}
+    for i, name in enumerate(input_node):
+        shape_dict[name] = input_data[i].shape
+        dtype_dict[name] = input_data[i].dtype.name
+
+    mod, params = relay.frontend.from_tflite(
+        tflite_model, shape_dict=shape_dict, dtype_dict=dtype_dict
+    )
+
+    return mod, params
+
+
+def _change_ndarray_layout(arr, src_layout, dst_layout):
+    """Makes a copy of an ndarray, reshaping it to a new data layout.
+
+    Parameter
+    ---------
+    arr : numpy.ndarray
+        The ndarray to be reformatted.
+
+    src_layout : str
+        The current layout of the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    dst_layout : str
+        The desired layout of new the Relay constant. Must be alphabetic (e.g. NHWC
+        or OIHW, but not NCHW2c).
+
+    Returns
+    -------
+    dst_shape : numpy.ndarray
+        A copy of the ndarray with the new layout.
+    """
+    assert src_layout.isalpha() and dst_layout.isalpha()
+    axis_order = [src_layout.index(c) for c in dst_layout]
+    return np.transpose(arr, axis_order)
+
+
+@tvm.testing.requires_package("tflite")
+@tvm.testing.requires_cmsisnn
+@pytest.mark.parametrize("test_runner", [AOT_CORSTONE300_RUNNER, AOT_USMP_CORSTONE300_RUNNER])
+def test_external_calls(test_runner):
+    """Download a small network and partition for CMSIS-NN to test forward declarations for external
+    calls outside of __tvm_main__."""
+    # download the model
+    base_url = (
+        "https://github.com/ARM-software/ML-zoo/raw/"
+        "48a22ee22325d15d2371a6df24eb7d67e21dcc97"
+        "/models/keyword_spotting/cnn_small/tflite_int8"
+    )
+    file_to_download = "cnn_s_quantized.tflite"
+    file_saved = "cnn_s_quantized_15Dec2021.tflite"
+    model_file = download_testdata("{}/{}".format(base_url, file_to_download), file_saved)
+
+    with open(model_file, "rb") as f:
+        tflite_model_buf = f.read()
+
+    input_shape = (1, 490)
+    dtype = "int8"
+    in_min, in_max = get_range_for_dtype_str(dtype)
+    rng = np.random.default_rng(12345)
+    input_data = rng.integers(in_min, high=in_max, size=input_shape, dtype=dtype)
+
+    orig_mod, params = _convert_to_relay(tflite_model_buf, input_data, "input")
+    cmsisnn_mod = cmsisnn.partition_for_cmsisnn(orig_mod, params)
+
+    # validate CMSIS-NN output against CPU output
+    interface_api = "c"
+    use_unpacked_api = True
+    inputs = {"input": input_data}
+    params = {}
+    output_list = generate_ref_data(orig_mod["main"], inputs, params)
+    compiled_models = compile_models(
+        AOTTestModel(
+            module=cmsisnn_mod,
+            inputs=inputs,
+            outputs=output_list,
+            params=None,
+            output_tolerance=1,
+        ),
+        interface_api,
+        use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    # Forward function declaration increases the number of times a function name appears under
+    # __tvm__main. Validate this frequency for native, offloaded and allocation functions.
+    lib_mod = compiled_models[0].executor_factory.lib.imported_modules[0]
+    main_source = lib_mod.get_source()
+    assert (
+        main_source.count("TVMBackendAllocWorkspace") == 3
+        or main_source.count("TVMBackendAllocWorkspace") == 0
+    )
+    assert main_source.count("tvmgen_default_fused_reshape") == 2
+    assert main_source.count("tvmgen_default_cmsis_nn_main") == 12
+    cmsisnn_source = lib_mod.imported_modules[0].get_source()
+    assert cmsisnn_source.count("arm_convolve_wrapper") == 1
+    assert cmsisnn_source.count("arm_fully_connected") == 3
+    assert cmsisnn_source.count("arm_softmax") == 1
+
+
+@parametrize_aot_options
+def test_internal_calls(interface_api, use_unpacked_api, test_runner):
+    """Test for all internal function calls. No forward declarations are expected here."""
+    dtype = "float32"
+    groups = 32
+    weight_shape = 1
+    ishape = (1, 32, 14, 14)
+    wshape = (32, weight_shape, 3, 3)
+    pass_config = {"tir.usmp.enable": True}
+    test_runner = AOTTestRunner(
+        makefile=test_runner.makefile,
+        prologue=test_runner.prologue,
+        epilogue=test_runner.epilogue,
+        includes=test_runner.includes,
+        parameters=test_runner.parameters,
+        pass_config=pass_config,
+    )
+
+    data0 = relay.var("data", shape=ishape, dtype=dtype)
+    weight0 = relay.var("weight", shape=wshape, dtype=dtype)
+    out = relay.nn.conv2d(data0, weight0, kernel_size=(3, 3), padding=(1, 1), groups=groups)
+    main_f = relay.Function([data0, weight0], out)
+    mod = tvm.IRModule()
+    mod["main"] = main_f
+    mod = tvm.relay.transform.InferType()(mod)
+
+    i_data = np.random.uniform(0, 1, ishape).astype(dtype)
+    w1_data = np.random.uniform(0, 1, wshape).astype(dtype)
+
+    inputs = OrderedDict([("data", i_data), ("weight", w1_data)])
+
+    output_list = generate_ref_data(mod, inputs)
+    compiled_models = compile_models(
+        models=AOTTestModel(module=mod, inputs=inputs, outputs=output_list),
+        interface_api=interface_api,
+        use_unpacked_api=use_unpacked_api,
+        pass_config=test_runner.pass_config,
+    )
+
+    lib_mod = compiled_models[0].executor_factory.lib.imported_modules[0]
+    main_source = lib_mod.get_source()
+    assert main_source.count("tvmgen_default_fused_nn_contrib_depthwise_conv2d_NCHWc") == 2
+    assert main_source.count("tvmgen_default_fused_layout_transform") == 6
+
+
+@skip_if_no_reference_system
+@tvm.testing.requires_cmsisnn
+def test_tensorized_calls():
+    """Test a subgraph with a mix of internal and tensorized calls."""
+    data_shape, kernel_size, num_filter, groups, strides, padding, dilation = (
+        (1, 32, 32, 16),
+        (3, 3),
+        16,
+        1,
+        1,
+        (0, 2, 2, 0),
+        1,
+    )
+    in_dtype = "int8"
+    data_layout = "NHWC"
+    kernel_layout = "HWOI"
+    ref_kernel_layout = "HWIO"
+    out_layout = "NHWC"
+    schedule_name = "conv2d_nhwc_dsp.arm_cpu"
+
+    ref_input_data = np.random.randint(low=-128, high=127, size=data_shape, dtype=in_dtype)
+    ref_input_var = relay.var("input", relay.TensorType(data_shape, in_dtype))  # NHWC layout
+    kernel_shape = (*kernel_size, data_shape[-1] // groups, num_filter)  # HWIO layout
+    ref_kernel_data = np.random.randint(low=-10, high=10, size=kernel_shape, dtype=in_dtype)
+
+    ref_relay_op = relay.op.nn.conv2d(
+        ref_input_var,
+        relay.const(_change_ndarray_layout(ref_kernel_data, "HWIO", ref_kernel_layout)),
+        kernel_size=kernel_size,
+        strides=strides,
+        padding=padding,
+        groups=groups,
+        dilation=(dilation, dilation),
+        data_layout="NHWC",
+        kernel_layout=ref_kernel_layout,
+        out_dtype="int32",
+        out_layout="NHWC",
+    )
+    ref_module = tvm.IRModule.from_expr(relay.Function([ref_input_var], ref_relay_op))
+    ref_outputs = generate_ref_data(ref_module, {"input": ref_input_data})
+
+    # Reshape output dictionary to match out_layout
+    assert len(ref_outputs) == 1
+    output_tensor_name, output_tensor = next(iter(ref_outputs.items()))
+    ref_outputs[output_tensor_name] = _change_ndarray_layout(output_tensor, "NHWC", out_layout)
+
+    test_input_data = _change_ndarray_layout(ref_input_data, "NHWC", data_layout)
+    test_input_var = relay.var("input", relay.TensorType(test_input_data.shape, in_dtype))
+    test_kernel_data = _change_ndarray_layout(ref_kernel_data, "HWIO", kernel_layout)
+
+    test_relay_op = relay.op.nn.conv2d(
+        test_input_var,
+        relay.const(test_kernel_data),
+        kernel_size=kernel_size,
+        strides=strides,
+        padding=padding,
+        groups=groups,
+        dilation=(dilation, dilation),
+        data_layout=data_layout,
+        kernel_layout=kernel_layout,
+        out_dtype="int32",
+        out_layout=out_layout,
+    )
+    test_function = relay.Function([test_input_var], test_relay_op)
+    test_model = AOTTestModel(
+        module=tvm.IRModule.from_expr(test_function),
+        inputs={"input": test_input_data},
+        outputs=ref_outputs,
+    )
+    compiled_models = compile_models(
+        test_model,
+        interface_api="c",
+        use_unpacked_api=True,
+        pass_config=AOT_CORSTONE300_RUNNER.pass_config,
+        target=f"c -keys=arm_cpu -mcpu=cortex-m7",
+        schedule_name=schedule_name,
+    )
+
+    lib_mod = compiled_models[0].executor_factory.lib.imported_modules[0]
+    main_source = lib_mod.get_source()
+    assert main_source.count("tvmgen_default_fused_nn_conv2d") == 2
+    assert main_source.count("gemm_") == 13

Review Comment:
   Noted. 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.

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

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