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/01 17:32:50 UTC

[GitHub] [tvm] quic-sanirudh opened a new pull request, #13256: [Hexagon] Add HVX quant conv2d implementation

quic-sanirudh opened a new pull request, #13256:
URL: https://github.com/apache/tvm/pull/13256

   This patch adds a new HVX intrinsic implementation to perform quantized convolution.
   
   It assumes that the qnn.conv2d relay op is not
   canonicalized and all the quantization parameters (scales and zero points) are passed into the intrinsic implementation.
   
   It also uses the fixed point computation function defined in hexagon topi utils to compute a fixed point (combined) scale which is used to perform the final requantization before returning the quantized output.


-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034657546


##########
src/runtime/hexagon/ops/conv2d_quant_hvx.cc:
##########
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <hexagon_types.h>
+#include <hvx_hexagon_protos.h>
+#include <inttypes.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/device_api.h>
+
+#include "conv2d.h"
+
+extern "C" int conv2d_packed_quant(TVMValue* args, int* type_codes, int num_args, TVMValue* out_val,
+                                   int out_code, void* res_handle);
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+inline uint8_t* getElementPtr_int8(int block_out_y, int block_out_x, int block_out_c, int yi,
+                                   int xi, int ci, const DLTensor& block) {
+  auto block_ptr =
+      tvm::runtime::hexagon::conv_utils::nhwc_at(block, 0, block_out_y, block_out_x, block_out_c);
+  auto block_offset = yi * 256 + xi * 32 + ci;

Review Comment:
   I'll do that @janetsc thanks. Right now they're assuming the activation blocks to be `8x8x32`.



-- 
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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1304723892

   @tvm-bot rerun


-- 
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] cconvey commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {

Review Comment:
   Is the addition of `inline` here (and elsewhere in the PR) necessary?
   
   From https://en.cppreference.com/w/cpp/language/constexpr:
   > A constexpr specifier used in a function or [static](https://en.cppreference.com/w/cpp/language/static) data member (since C++17) declaration implies inline.
   



-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1016416568


##########
tests/python/contrib/test_hexagon/topi/test_conv2d_quant_intrin.py:
##########
@@ -0,0 +1,261 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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 quantized conv2d HVX intrinsic implementation"""
+
+import numpy as np
+
+import tvm
+import tvm.contrib.hexagon
+from tvm.topi.hexagon.utils import get_fixed_point_value
+from tvm.topi.testing import conv2d_nhwc_python
+
+from ..infrastructure import get_hexagon_target, quantize_np
+
+
+def build_conv2d(target):
+    """Build and return the conv2d IRModule that calls the intrinsic implementation"""
+    act_n, act_h, act_w, act_c = (
+        tvm.te.var("an"),
+        tvm.te.var("ah"),
+        tvm.te.var("aw"),
+        tvm.te.var("ac"),
+    )
+    filt_h, filt_w, filt_o = tvm.te.var("filt_h"), tvm.te.var("filt_w"), tvm.te.var("filt_o")
+    act_scale, act_zp = tvm.te.var("act_scale", dtype="float32"), tvm.te.var("act_zp")
+    wgt_scale, wgt_zp = tvm.te.var("wgt_scale", dtype="float32"), tvm.te.var("wgt_zp")
+    out_scale, out_zp = tvm.te.var("out_scale", dtype="float32"), tvm.te.var("out_zp")
+    fixed_final_scale, scale_factor = tvm.te.var("fixed_final_scale", dtype="int32"), tvm.te.var(
+        "scale_factor"
+    )
+    stride_h, stride_w = tvm.te.var("stride_h"), tvm.te.var("stride_w")
+
+    act_flat = tvm.te.placeholder(
+        shape=(act_n, act_h, act_w, act_c), dtype="uint8", name="act_flat"
+    )
+    wgt_flat = tvm.te.placeholder(
+        shape=(filt_h, filt_w, act_c, filt_o), dtype="int8", name="wgt_flat"
+    )
+
+    out_flat = tvm.te.extern(
+        shape=(act_n, (act_h - filt_h) // stride_h + 1, (act_w - filt_w) // stride_w + 1, filt_o),
+        inputs=[act_flat, wgt_flat],
+        fcompute=lambda ins, outs: tvm.tir.call_cpacked(
+            "conv2d_packed_quant",  # Function from TVM runtime
+            ins[0],
+            ins[1],
+            act_scale,
+            act_zp,
+            wgt_scale,
+            wgt_zp,
+            out_scale,
+            out_zp,
+            stride_h,
+            stride_w,
+            fixed_final_scale,
+            scale_factor,
+            outs[0],
+            tvm.runtime.const(0),  # resource_handle (unused)
+        ),
+        dtype="uint8",
+    )
+
+    s = tvm.te.create_schedule(out_flat.op)
+
+    func_name = "conv2d_quant_hvx"
+    module = tvm.build(
+        s,
+        [
+            act_flat,
+            wgt_flat,
+            act_scale,
+            act_zp,
+            wgt_scale,
+            wgt_zp,
+            out_scale,
+            out_zp,
+            stride_h,
+            stride_w,
+            fixed_final_scale,
+            scale_factor,
+            out_flat,
+        ],
+        target=target,
+        name=func_name,
+    )
+
+    return module
+
+
+def gen_config(params):
+    """Utility function to generate useful ids for shape_parameters"""
+
+    dims = lambda vals: "x".join(map(str, vals))
+
+    config = {}
+    for param in params:
+        act_shape, wgt_shape, inp_stride = param
+        name = f"nhwc{dims(act_shape)}-hwio{dims(wgt_shape)}-stride{dims(inp_stride)}"
+        config[name] = param
+
+    return config
+
+
+class TestQuantConv2dIntrin:
+    """Test Quantized Conv2d Intrin class"""
+
+    shape_parameters = [
+        [
+            (1, 5, 5, 33),
+            (3, 3, 33, 33),
+            (1, 1),
+        ],
+        [
+            (1, 9, 8, 64),
+            (3, 3, 64, 64),
+            (1, 1),
+        ],
+        [
+            (1, 11, 16, 64),
+            (3, 3, 64, 32),
+            (1, 1),
+        ],
+        [
+            (1, 24, 8, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 4, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 5, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 6, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 7, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 8, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 9, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 10, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 11, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 4, 5),
+            (3, 3, 5, 3),
+            (1, 1),
+        ],
+    ]
+
+    config = gen_config(shape_parameters)
+    act_shape, wgt_shape, inp_stride = tvm.testing.parameters(*config.values(), ids=config.keys())
+    inp_offset = tvm.testing.parameter((0, 0), ids=["offset0x0"])
+
+    @tvm.testing.requires_hexagon
+    def test_conv2d_quant(self, act_shape, wgt_shape, inp_stride, hexagon_session):
+        """Test quantized conv2d intrinsic implementation"""
+        assert act_shape[3] == wgt_shape[2]
+
+        # Currently, input offset does not affect the output shape
+        def get_out_shape(ash, wsh, inp_stride):
+            assert ash[3] == wsh[2]
+            osh = (
+                ash[0],
+                (ash[1] - wsh[0]) // inp_stride[0] + 1,
+                (ash[2] - wsh[1]) // inp_stride[1] + 1,
+                wsh[3],
+            )
+            assert tvm.tir.all([x > 0 for x in osh])
+            return osh
+
+        act_f = np.random.uniform(-1.5, 1.0, size=act_shape).astype("float32")
+        wgt_f = np.random.uniform(-1.5, 1.0, size=wgt_shape).astype("float32")
+
+        # Quanize activations using onnxruntime
+        act_q, act_scale, act_zp = quantize_np(act_f, dtype="uint8")
+        act_q = act_q.reshape(act_f.shape)
+
+        # Quanize weights using onnxruntime
+        wgt_q, wgt_scale, wgt_zp = quantize_np(wgt_f, dtype="int8")
+        wgt_q = wgt_q.reshape(wgt_f.shape)
+
+        # Generate reference output
+        ref_out = conv2d_nhwc_python(act_f, wgt_f, stride=inp_stride, padding="VALID")
+
+        ref_out_q, out_scale, out_zp = quantize_np(ref_out, dtype="uint8")
+        ref_out_q = ref_out_q.reshape(ref_out.shape)
+
+        final_scale = act_scale * wgt_scale / out_scale
+        fixed_final_scale, scale_factor = get_fixed_point_value(final_scale)

Review Comment:
   Hi @ibsidorenko, thanks for the review. The `int16` dtype was chosen so that the arithmetic for re-quantization can happen in `int32`, which reduces the number of instructions, but yes the accuracy could be affected. I haven't tested this on real world models yet, but that was the reason for setting a very tight `rtol`/`atol` values for assertion in the test case.
   
   I also tried to break the accuracy of `int16` fixed point computation by initializing the random inputs to extreme ranges and getting the scale values in the order of `0.0001` to `1000` (which was well beyond any scale values I saw in real life models), and the test still passed with the expected accuracy. 
   
   I plan to verify this on real world models and see how the accuracy is affected (if at all) and if needed I can update the patch to use int32 fixed point values instead.



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

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

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


[GitHub] [tvm] mehrdadh merged pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034655362


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc
+  return y << 8 | x << 5 | c;
+}
+
+inline constexpr int hwio_to_sm_8b(int width, int y, int x, int i, int o) {
+  // Map y,x,i,o coordinates within a chunk (assuming the origin at the
+  // top-left spatial corner) to the offset (in 8-bit elements) from the
+  // beginning of the chunk in spatial-major layout.
+  // Spatial mask: p..piiioooooii, where p..p are position bits.
+  int p = y * width + (width - 1 - x);
+  return p << 10 | (i & 0x1c) << 5 | o << 2 | (i & 3);

Review Comment:
   Same comment as above. I can probably add asserts if we can disable them for release builds later



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

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

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


[GitHub] [tvm] ibsidorenko commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
tests/python/contrib/test_hexagon/topi/test_conv2d_quant_intrin.py:
##########
@@ -0,0 +1,261 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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 quantized conv2d HVX intrinsic implementation"""
+
+import numpy as np
+
+import tvm
+import tvm.contrib.hexagon
+from tvm.topi.hexagon.utils import get_fixed_point_value
+from tvm.topi.testing import conv2d_nhwc_python
+
+from ..infrastructure import get_hexagon_target, quantize_np
+
+
+def build_conv2d(target):
+    """Build and return the conv2d IRModule that calls the intrinsic implementation"""
+    act_n, act_h, act_w, act_c = (
+        tvm.te.var("an"),
+        tvm.te.var("ah"),
+        tvm.te.var("aw"),
+        tvm.te.var("ac"),
+    )
+    filt_h, filt_w, filt_o = tvm.te.var("filt_h"), tvm.te.var("filt_w"), tvm.te.var("filt_o")
+    act_scale, act_zp = tvm.te.var("act_scale", dtype="float32"), tvm.te.var("act_zp")
+    wgt_scale, wgt_zp = tvm.te.var("wgt_scale", dtype="float32"), tvm.te.var("wgt_zp")
+    out_scale, out_zp = tvm.te.var("out_scale", dtype="float32"), tvm.te.var("out_zp")
+    fixed_final_scale, scale_factor = tvm.te.var("fixed_final_scale", dtype="int32"), tvm.te.var(
+        "scale_factor"
+    )
+    stride_h, stride_w = tvm.te.var("stride_h"), tvm.te.var("stride_w")
+
+    act_flat = tvm.te.placeholder(
+        shape=(act_n, act_h, act_w, act_c), dtype="uint8", name="act_flat"
+    )
+    wgt_flat = tvm.te.placeholder(
+        shape=(filt_h, filt_w, act_c, filt_o), dtype="int8", name="wgt_flat"
+    )
+
+    out_flat = tvm.te.extern(
+        shape=(act_n, (act_h - filt_h) // stride_h + 1, (act_w - filt_w) // stride_w + 1, filt_o),
+        inputs=[act_flat, wgt_flat],
+        fcompute=lambda ins, outs: tvm.tir.call_cpacked(
+            "conv2d_packed_quant",  # Function from TVM runtime
+            ins[0],
+            ins[1],
+            act_scale,
+            act_zp,
+            wgt_scale,
+            wgt_zp,
+            out_scale,
+            out_zp,
+            stride_h,
+            stride_w,
+            fixed_final_scale,
+            scale_factor,
+            outs[0],
+            tvm.runtime.const(0),  # resource_handle (unused)
+        ),
+        dtype="uint8",
+    )
+
+    s = tvm.te.create_schedule(out_flat.op)
+
+    func_name = "conv2d_quant_hvx"
+    module = tvm.build(
+        s,
+        [
+            act_flat,
+            wgt_flat,
+            act_scale,
+            act_zp,
+            wgt_scale,
+            wgt_zp,
+            out_scale,
+            out_zp,
+            stride_h,
+            stride_w,
+            fixed_final_scale,
+            scale_factor,
+            out_flat,
+        ],
+        target=target,
+        name=func_name,
+    )
+
+    return module
+
+
+def gen_config(params):
+    """Utility function to generate useful ids for shape_parameters"""
+
+    dims = lambda vals: "x".join(map(str, vals))
+
+    config = {}
+    for param in params:
+        act_shape, wgt_shape, inp_stride = param
+        name = f"nhwc{dims(act_shape)}-hwio{dims(wgt_shape)}-stride{dims(inp_stride)}"
+        config[name] = param
+
+    return config
+
+
+class TestQuantConv2dIntrin:
+    """Test Quantized Conv2d Intrin class"""
+
+    shape_parameters = [
+        [
+            (1, 5, 5, 33),
+            (3, 3, 33, 33),
+            (1, 1),
+        ],
+        [
+            (1, 9, 8, 64),
+            (3, 3, 64, 64),
+            (1, 1),
+        ],
+        [
+            (1, 11, 16, 64),
+            (3, 3, 64, 32),
+            (1, 1),
+        ],
+        [
+            (1, 24, 8, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 4, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 5, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 6, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 7, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 8, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 9, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 10, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 11, 3),
+            (3, 3, 3, 3),
+            (1, 1),
+        ],
+        [
+            (1, 4, 4, 5),
+            (3, 3, 5, 3),
+            (1, 1),
+        ],
+    ]
+
+    config = gen_config(shape_parameters)
+    act_shape, wgt_shape, inp_stride = tvm.testing.parameters(*config.values(), ids=config.keys())
+    inp_offset = tvm.testing.parameter((0, 0), ids=["offset0x0"])
+
+    @tvm.testing.requires_hexagon
+    def test_conv2d_quant(self, act_shape, wgt_shape, inp_stride, hexagon_session):
+        """Test quantized conv2d intrinsic implementation"""
+        assert act_shape[3] == wgt_shape[2]
+
+        # Currently, input offset does not affect the output shape
+        def get_out_shape(ash, wsh, inp_stride):
+            assert ash[3] == wsh[2]
+            osh = (
+                ash[0],
+                (ash[1] - wsh[0]) // inp_stride[0] + 1,
+                (ash[2] - wsh[1]) // inp_stride[1] + 1,
+                wsh[3],
+            )
+            assert tvm.tir.all([x > 0 for x in osh])
+            return osh
+
+        act_f = np.random.uniform(-1.5, 1.0, size=act_shape).astype("float32")
+        wgt_f = np.random.uniform(-1.5, 1.0, size=wgt_shape).astype("float32")
+
+        # Quanize activations using onnxruntime
+        act_q, act_scale, act_zp = quantize_np(act_f, dtype="uint8")
+        act_q = act_q.reshape(act_f.shape)
+
+        # Quanize weights using onnxruntime
+        wgt_q, wgt_scale, wgt_zp = quantize_np(wgt_f, dtype="int8")
+        wgt_q = wgt_q.reshape(wgt_f.shape)
+
+        # Generate reference output
+        ref_out = conv2d_nhwc_python(act_f, wgt_f, stride=inp_stride, padding="VALID")
+
+        ref_out_q, out_scale, out_zp = quantize_np(ref_out, dtype="uint8")
+        ref_out_q = ref_out_q.reshape(ref_out.shape)
+
+        final_scale = act_scale * wgt_scale / out_scale
+        fixed_final_scale, scale_factor = get_fixed_point_value(final_scale)

Review Comment:
   Hi, @quic-sanirudh ! Thank you for this PR, rather interesting work. Just one small question:
   As I see for Hexagon target we use int16 dtype to represent  fixed point values (`dtype `param in `get_fixed_point_value`). But in TVM we use int32 dtype for that (for example for scale parameter in requantize). Can this somehow affect the accuracy of the real life quantized models?



-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1015287493


##########
tests/python/contrib/test_hexagon/topi/test_conv2d_quant_intrin.py:
##########
@@ -0,0 +1,262 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   I've fixed the line issue, 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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034912923


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {

Review Comment:
   ah okay, I did not realize this change was made. Looks like an addition that was inserted by working with the downstream repo, where this `inline` probably exists. I'll remove it, it 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] cconvey commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
cmake/modules/Hexagon.cmake:
##########
@@ -178,6 +178,15 @@ if(BUILD_FOR_HEXAGON)
     "${TVMRT_SOURCE_DIR}/hexagon/ops/*.cc"
   )
 
+  include_directories(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops"
+  )
+
+  set_source_files_properties(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops/conv2d_quant_hvx.cc"
+    PROPERTIES COMPILE_FLAGS "-mhvx"

Review Comment:
   Are we confident that `-mhvx` is supported by all of the compilers that might build this code?
   
   I'm assuming that _typically_ the clang provided by Hexagon Toolchain will be used.  But I'm a little fuzzy about the intended level of support for other compilers, e.g. a user-supplied build of Clang/LLVM.



##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -144,7 +207,42 @@ void blockize_hwc_16b(void* out, void* inp_flat, int height, int width, int dept
  * @param width
  * @param depth
  */
-void deblockize_hwc_16b(void* out_flat, void* inp, int height, int width, int depth);
+template <typename T, int block_height, int block_width, int block_depth>
+void deblockize_hwc(void* out_flat, void* inp, int height, int width, int depth) {

Review Comment:
   Would it make sense for the type of `inp` to be `const void*`?



##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -133,7 +155,48 @@ inline uintptr_t hwio_at(const DLTensor& f, int y, int x, int i, int o) {
  * @param width
  * @param depth
  */
-void blockize_hwc_16b(void* out, void* inp_flat, int height, int width, int depth);
+template <typename T, int block_height, int block_width, int block_depth>
+void blockize_hwc(void* out, void* inp_flat, int height, int width, int depth) {

Review Comment:
   Would it make sense for `inp_flat`'s type to be `const void*` rather than `void*`?
   
   This is probably a bit of a stylistic choice; I just figured I'd ask.



##########
cmake/modules/Hexagon.cmake:
##########
@@ -178,6 +178,15 @@ if(BUILD_FOR_HEXAGON)
     "${TVMRT_SOURCE_DIR}/hexagon/ops/*.cc"
   )
 
+  include_directories(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops"
+  )
+
+  set_source_files_properties(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops/conv2d_quant_hvx.cc"
+    PROPERTIES COMPILE_FLAGS "-mhvx"

Review Comment:
   Would it make sense to update [src/runtime/hexagon/README.md](https://github.com/apache/tvm/blob/15ee9bb5757915c73569f3ebdb5e52a4312663aa/src/runtime/hexagon/README.md) to clarify the version(s) of LLVM that support flags like `-mhvx`?
   
   Or alternatively, use CMake's [CheckCXXCompilerFlag](https://cmake.org/cmake/help/latest/module/CheckCXXCompilerFlag.html) function to see if `-mhvx` is supported, and only use that flag if it is?



##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   I'm pretty sure we can rely on `assert` being disabled when `CMAKE_BUILD_TYPE=Release`.  See https://stackoverflow.com/questions/34302265/does-cmake-build-type-release-imply-dndebug.



-- 
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] janetsc commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   Another option is to check the loop bounds to make sure y, x and c can't get bigger than can be expressed.



-- 
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 #13256: [Hexagon] Add HVX quant conv2d implementation

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

   <!---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-->
    * cc @mehrdadh <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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1306697787

   > > @driazati @mehrdadh The CI is stuck. Could you please let me know if there's any way to restart the CI. Would pining the tvm-bot work?
   > 
   > CI is finished now, so it should be good. I've been seeing some similar queueing issues (probably Jenkins wasn't able to spin up a machine to run the jobs for some reason), I opened #13312 and will be looking into this today
   
   Great, thanks a lot for the help. 


-- 
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] driazati commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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

   > @driazati @mehrdadh The CI is stuck. Could you please let me know if there's any way to restart the CI. Would pining the tvm-bot work?
   
   CI is finished now, so it should be good. I've been seeing some similar queueing issues (probably Jenkins wasn't able to spin up a machine to run the jobs for some reason), I opened #13312 and will be looking into this today


-- 
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] janetsc commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -133,7 +155,48 @@ inline uintptr_t hwio_at(const DLTensor& f, int y, int x, int i, int o) {
  * @param width
  * @param depth
  */
-void blockize_hwc_16b(void* out, void* inp_flat, int height, int width, int depth);
+template <typename T, int block_height, int block_width, int block_depth>
+void blockize_hwc(void* out, void* inp_flat, int height, int width, int depth) {
+  int (*index_func)(int, int, int);
+  if constexpr (std::is_same_v<T, uint8_t>)
+    index_func = yxc_to_sm_8b;
+  else if constexpr (std::is_same_v<T, uint16_t>)
+    index_func = yxc_to_sm_16b;
+  else
+    LOG_ERROR << "blockize_hwc is only supported for uint8_t and uint16_t types";
+
+  auto inp_data = static_cast<T*>(inp_flat);
+  auto out_data = static_cast<uintptr_t*>(out);
+  const int stride_x = depth;
+  const int stride_y = stride_x * width;
+
+  for (int cy = 0; cy < height; cy += block_height) {
+    for (int cx = 0; cx < width; cx += block_width) {
+      for (int cc = 0; cc < depth; cc += block_depth) {
+        auto block = reinterpret_cast<T*>(*out_data++);
+        int max_y = std::min(block_height, height - cy);
+        int max_x = std::min(block_width, width - cx);
+        int max_c = std::min(block_depth, depth - cc);

Review Comment:
   This is an example where you could check to see that max_y, max_x and max_c are in bounds for index_func. 



-- 
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] janetsc commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   Agreed - this is much safer!



-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034650897


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   I consciously avoided the checks here because these functions are used for indexing within the innermost loops and need to be really fast. I actually was planning to remove the check from the above `yxc_to_sm_16b` function as well.
   
   I thought I had was to add the `assert` statements and then disable them for release builds with `#define NDEBUG`. Not sure if there's a better solution.



-- 
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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1310278577

   @csullivan @cconvey When you get a chance, could you review this PR. This is a quantized conv2d similar to the fp16 conv2d that I wrote earlier.


-- 
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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1305895423

   @driazati @mehrdadh The CI is stuck. Could you please let me know if there's any way to restart the CI. Would pining the tvm-bot work?


-- 
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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1318072797

   @csullivan @cconvey Could you please help in reviewing this patch when you get a chance, 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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1329079539

   @csullivan @cconvey Could you please review this PR or suggest someone who could be the right person to review it. 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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034826123


##########
cmake/modules/Hexagon.cmake:
##########
@@ -178,6 +178,15 @@ if(BUILD_FOR_HEXAGON)
     "${TVMRT_SOURCE_DIR}/hexagon/ops/*.cc"
   )
 
+  include_directories(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops"
+  )
+
+  set_source_files_properties(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops/conv2d_quant_hvx.cc"
+    PROPERTIES COMPILE_FLAGS "-mhvx"

Review Comment:
   Thanks for the review @cconvey.
   
   I can add the details in the README or add a CMake check, but the `-mhvx` flag was added to clang all the way back in 2017 in [LLVM 6.0 release](https://releases.llvm.org/6.0.0/docs/ReleaseNotes.html#changes-to-the-hexagon-target) if not earlier, which predates the entire TVM project, so we can also probably assume safely that the `-mhvx` flag will be available for practically anyone building the TVM project now.
   
   If you think it might still be better to add the check or the README change, please let me know which one you think makes more sense and I can make that change. 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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034828798


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -144,7 +207,42 @@ void blockize_hwc_16b(void* out, void* inp_flat, int height, int width, int dept
  * @param width
  * @param depth
  */
-void deblockize_hwc_16b(void* out_flat, void* inp, int height, int width, int depth);
+template <typename T, int block_height, int block_width, int block_depth>
+void deblockize_hwc(void* out_flat, void* inp, int height, int width, int depth) {

Review Comment:
   I'll add the `const void*`, it 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] janetsc commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   Add a check to make sure only the bits we expect are set in the inputs - for y and x only the lowest 3 bits and c only 5 bits



##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc
+  return y << 8 | x << 5 | c;
+}
+
+inline constexpr int hwio_to_sm_8b(int width, int y, int x, int i, int o) {
+  // Map y,x,i,o coordinates within a chunk (assuming the origin at the
+  // top-left spatial corner) to the offset (in 8-bit elements) from the
+  // beginning of the chunk in spatial-major layout.
+  // Spatial mask: p..piiioooooii, where p..p are position bits.
+  int p = y * width + (width - 1 - x);
+  return p << 10 | (i & 0x1c) << 5 | o << 2 | (i & 3);

Review Comment:
   Suggest similar bounds checking here.



##########
src/runtime/hexagon/ops/conv2d_quant_hvx.cc:
##########
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <hexagon_types.h>
+#include <hvx_hexagon_protos.h>
+#include <inttypes.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/device_api.h>
+
+#include "conv2d.h"
+
+extern "C" int conv2d_packed_quant(TVMValue* args, int* type_codes, int num_args, TVMValue* out_val,
+                                   int out_code, void* res_handle);
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+inline uint8_t* getElementPtr_int8(int block_out_y, int block_out_x, int block_out_c, int yi,
+                                   int xi, int ci, const DLTensor& block) {
+  auto block_ptr =
+      tvm::runtime::hexagon::conv_utils::nhwc_at(block, 0, block_out_y, block_out_x, block_out_c);
+  auto block_offset = yi * 256 + xi * 32 + ci;

Review Comment:
   The same comment applies to all constants below as well.



##########
src/runtime/hexagon/ops/conv2d_quant_hvx.cc:
##########
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <hexagon_types.h>
+#include <hvx_hexagon_protos.h>
+#include <inttypes.h>
+#include <tvm/runtime/c_runtime_api.h>
+#include <tvm/runtime/device_api.h>
+
+#include "conv2d.h"
+
+extern "C" int conv2d_packed_quant(TVMValue* args, int* type_codes, int num_args, TVMValue* out_val,
+                                   int out_code, void* res_handle);
+
+namespace tvm {
+namespace runtime {
+namespace hexagon {
+inline uint8_t* getElementPtr_int8(int block_out_y, int block_out_x, int block_out_c, int yi,
+                                   int xi, int ci, const DLTensor& block) {
+  auto block_ptr =
+      tvm::runtime::hexagon::conv_utils::nhwc_at(block, 0, block_out_y, block_out_x, block_out_c);
+  auto block_offset = yi * 256 + xi * 32 + ci;

Review Comment:
   Suggest defining consts for these.  Are they derived from the supported shape?



-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1035023792


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   I've added the asserts directly inside the index functions that would be disabled with Release builds.
   
   I thought about adding it in the outer loops as you suggested, but that anyways is guaranteed with the current code as `block_height/block_width/block_depth` is expected to be the size of the blocks and for other uses of the index functions, it is the responsibility of the caller anyway.



-- 
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] quic-sanirudh commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on PR #13256:
URL: https://github.com/apache/tvm/pull/13256#issuecomment-1303914219

   @csullivan @cconvey Gentle ping for a review. This is an initial quant implementation of conv2d, similar to the fp16 version that I wrote earlier.


-- 
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] quic-sanirudh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

Posted by GitBox <gi...@apache.org>.
quic-sanirudh commented on code in PR #13256:
URL: https://github.com/apache/tvm/pull/13256#discussion_r1034828402


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -133,7 +155,48 @@ inline uintptr_t hwio_at(const DLTensor& f, int y, int x, int i, int o) {
  * @param width
  * @param depth
  */
-void blockize_hwc_16b(void* out, void* inp_flat, int height, int width, int depth);
+template <typename T, int block_height, int block_width, int block_depth>
+void blockize_hwc(void* out, void* inp_flat, int height, int width, int depth) {

Review Comment:
   I agree with both, I'll add the asserts and the `const void*` for the arguments, 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] cconvey commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
cmake/modules/Hexagon.cmake:
##########
@@ -178,6 +178,15 @@ if(BUILD_FOR_HEXAGON)
     "${TVMRT_SOURCE_DIR}/hexagon/ops/*.cc"
   )
 
+  include_directories(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops"
+  )
+
+  set_source_files_properties(
+    "${TVMRT_SOURCE_DIR}/hexagon/ops/conv2d_quant_hvx.cc"
+    PROPERTIES COMPILE_FLAGS "-mhvx"

Review Comment:
   That makes total sense, I didn't realize `-mhvx` support went back that far.  I agree that there's no need for any additional documentation or checking.



-- 
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] janetsc commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -133,7 +155,48 @@ inline uintptr_t hwio_at(const DLTensor& f, int y, int x, int i, int o) {
  * @param width
  * @param depth
  */
-void blockize_hwc_16b(void* out, void* inp_flat, int height, int width, int depth);
+template <typename T, int block_height, int block_width, int block_depth>
+void blockize_hwc(void* out, void* inp_flat, int height, int width, int depth) {
+  int (*index_func)(int, int, int);
+  if constexpr (std::is_same_v<T, uint8_t>)
+    index_func = yxc_to_sm_8b;
+  else if constexpr (std::is_same_v<T, uint16_t>)
+    index_func = yxc_to_sm_16b;
+  else
+    LOG_ERROR << "blockize_hwc is only supported for uint8_t and uint16_t types";
+
+  auto inp_data = static_cast<T*>(inp_flat);
+  auto out_data = static_cast<uintptr_t*>(out);
+  const int stride_x = depth;
+  const int stride_y = stride_x * width;
+
+  for (int cy = 0; cy < height; cy += block_height) {
+    for (int cx = 0; cx < width; cx += block_width) {
+      for (int cc = 0; cc < depth; cc += block_depth) {
+        auto block = reinterpret_cast<T*>(*out_data++);
+        int max_y = std::min(block_height, height - cy);
+        int max_x = std::min(block_width, width - cx);
+        int max_c = std::min(block_depth, depth - cc);

Review Comment:
   This is an example where you could check to see that max_y, max_x and max_c are in bounds for index_func. 



-- 
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] janetsc commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
src/runtime/hexagon/ops/conv2d.h:
##########
@@ -75,15 +77,31 @@ inline void* to_ptr(uintptr_t v) { return reinterpret_cast<void*>(v); }
 
 inline uintptr_t to_uint(void* ptr) { return reinterpret_cast<uintptr_t>(ptr); }
 
-constexpr int xyc_to_sm_16b(int y, int x, int c) {
+inline constexpr int yxc_to_sm_16b(int y, int x, int c) {
   // Map y,x,c coordinates within a block to the offset (in 16-bit elements)
   // from the beginning of the block in spatial-major layout.
   // 10-bit spatial mask: yyyxcccccx
   assert(y >= 0 && x >= 0 && c >= 0);
   return y << 7 | (x & 2) << 5 | c << 1 | (x & 1);
 }
 
-constexpr int hwio_to_sm_16b(int width, int y, int x, int i, int o) {
+inline constexpr int yxc_to_sm_8b(int y, int x, int c) {
+  // Map y,x,c coordinates within a block to the offset (in 8-bit elements)
+  // from the beginning of the block in spatial-major layout.
+  // 10-bit spatial mask: yyyxxxccccc

Review Comment:
   Another option is to check the loop bounds in the caller to make sure y, x and c can't get bigger than can be expressed.  (And put a comment here to that effect - that it is the caller's responsibility to check on release builds.)



-- 
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] mehrdadh commented on a diff in pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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


##########
tests/python/contrib/test_hexagon/topi/test_conv2d_quant_intrin.py:
##########
@@ -0,0 +1,262 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   This file will have lint issues once https://github.com/apache/tvm/pull/13271 merged. Please fix them in the meantime



-- 
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] driazati commented on pull request #13256: [Hexagon] Add HVX quant conv2d implementation

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

   > @driazati @mehrdadh The CI is stuck. Could you please let me know if there's any way to restart the CI. Would pining the tvm-bot work?
   
   CI is finished now, so it should be good. I've been seeing some similar queueing issues (probably Jenkins wasn't able to spin up a machine to run the jobs for some reason), I opened #13312 and will be looking into this today


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