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/07/25 00:03:02 UTC

[GitHub] [tvm] cconvey opened a new pull request, #12169: WIP [hexagon][topi] add sliced max_pool2

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

   Add TOPI implementation of sliced max_pool2d.
   
   This is an early version of the implementation.
   It does use Hexagon-friendly memory layouts, but
   contains no significant Schedule primitives.
   


-- 
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 pull request #12169: [hexagon][topi] add sliced max_pool2

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

   @Lunderberg @csullivan : Any chance one of you could give this a review?


-- 
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 #12169: [hexagon][topi] add sliced max_pool2

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


##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op
+
+Please note the following assumptions made by the implementation:
+
+1) The input must be padded in advance to account for 'padding'. In addition,
+   both input and output must be padded as per the physical buffer layout.
+2) The current implementation assumes 'count_include_pad' to be 'True'. It can be
+   modified to support 'False' case but the element count for the pooling window
+   must be pre-computed and provided as an input to reduce the run-time overhead.
+3) 'padding' is ignored. It must be handled outside of the sliced op.
+4) Please note that this implementation will not work if the output includes any
+   physical layout related padding as it can result into out-of-bound access
+   for the input.
+"""
+
+from tvm import te
+from tvm import tir
+from ..utils import get_layout_transform_fn
+
+import io
+import sys
+from typing import *
+
+
+def validate_out_shape(out_shape, in_shape, kernel, stride, dilation):
+    """Validate output shape"""
+    _, oh, ow, _ = out_shape
+    _, ih, iw, _ = in_shape
+    kh, kw = kernel
+    sh, sw = stride
+    dh, dw = dilation
+    if ih < (oh - 1) * sh + dh * (kh - 1) + 1:
+        raise RuntimeError("Output height is too large")
+    if iw < (ow - 1) * sw + dw * (kw - 1) + 1:
+        raise RuntimeError("Output width is too large")
+
+
+def max_pool2d_compute(A, out_shape, kernel, stride, dilation):
+    """max_pool2d compute"""
+    kh, kw = kernel
+    rh = te.reduce_axis((0, kh), name="rh")
+    rw = te.reduce_axis((0, kw), name="rw")
+    ob, oh, ow, oc = out_shape
+    if isinstance(ob, int):
+        validate_out_shape(out_shape, A.shape, kernel, stride, dilation)
+
+    sh, sw = stride
+    dh, dw = dilation
+
+    Max = te.compute(
+        out_shape,
+        lambda b, h, w, c: te.max(
+            A[b, h * sh + dh * rh, w * sw + dw * rw, c].astype(A.dtype), axis=[rh, rw]
+        ),
+        name="max",
+    )
+    return Max
+
+
+def STIR_schedule_nhwc_8h2w32c2w(outs, ins, output_layout: str, input_layout: str):
+    """Schedule for input and output layout nhwc-8h2w32c2w"""
+    func = te.create_prim_func([ins, outs])
+    s = tir.Schedule(func)
+
+    # NOTE!!! This scheduling logic is a work in progress.
+    # It is not known to ultimately result in near-optimal Hexagon performance.
+    # The schedule below strives to implement these heuristics:
+    #
+    # - Each 2048-byte chunk of the output tensor should be visited only once, if possible.
+    #
+    # - The resulting object code should use Hexagon v69's HVX SIMD units if at all possible.
+    #   (The HVX SIMD registers are 2048 bytes long, so each "chunk" of the output tensor
+    #   fits exactly in a single HVX SIMD register.)
+
+    Max = s.get_block("max")
+
+    input_transform_fn = get_layout_transform_fn(input_layout)
+    output_transform_fn = get_layout_transform_fn(output_layout)
+
+    s.transform_layout(Max, ("read", 0), input_transform_fn)
+    s.transform_layout(Max, ("write", 0), output_transform_fn)
+
+    # Restructure the loop nestings to have this overall structure:
+    # (loop over different 2048-byte output-tensor chunks) : n, ho, wo, co   }- the first level of a two-level tensor layout
+    #    (loop within one 2048-byte output-tensor chunk) : hi, wio, ci, wii  }- the second level of a two-level tensor layout
+    #        (loop over reduction axes) : rh, rw                             }- loop over multiple elements of the input tensor
+    #
+    # Note: This schedule is a work in progress.  We *expect* that it's
+    # crucially important for the loops to have this relative ordering:
+    #    n ... ho ... wo ... co ... hi ... wio ... ci ... wii
+    # because it lets us visit each of the 2048-byte output chunks precisely once.
+
+    (
+        n,
+        h,
+        w,
+        c,
+        rh,
+        rw,
+    ) = s.get_loops(Max)
+
+    # Restructure the loops from NHWC to nhwc_8h2w32c2w, with loops for 'sum's reduction
+    # axes at the very end.
+    ho, hi = s.split(h, [None, 8])
+    wo, wi = s.split(w, [None, 4])
+    wio, wii = s.split(wi, [None, 2])
+    co, ci = s.split(c, [None, 32])
+    s.reorder(n, ho, wo, co, hi, wio, ci, wii, rh, rw)

Review Comment:
   From talking with @Lunderberg , I got the impression that some or all of these loop manipulations were expected to already be done by the earlier call to `transform_layout`.  
   
   TODO(cconvey): Figure out if my understanding was wrong, or there's a bug in transform_layout.



-- 
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] jverma-quic commented on a diff in pull request #12169: WIP [hexagon][topi] add sliced max_pool2

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


##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op

Review Comment:
   avg_pool2d -> max_pool2d



##########
tests/python/contrib/test_hexagon/topi/test_max_pool2d_slice.py:
##########
@@ -0,0 +1,407 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+import numpy as np
+from typing import *
+
+from tvm import te
+import tvm.testing
+from tvm.topi import testing
+from tvm.contrib.hexagon.build import HexagonLauncher
+from tvm.contrib.hexagon.session import Session
+import tvm.topi.hexagon.slice_ops as sl
+from ..infrastructure import allocate_hexagon_array, transform_numpy
+from ..pytest_util import (
+    get_numpy_dtype_info,
+    get_test_id,
+    get_multitest_ids,
+    create_populated_numpy_ndarray,
+    TensorContentConstant,
+    TensorContentRandom,
+    TensorContentDtypeMin,
+    TensorContentDtypeMax,
+    TensorContentSequentialCOrder,
+)
+
+
+@tvm.testing.fixture
+def input_np(input_shape, dtype: str, input_tensor_populator):
+    return create_populated_numpy_ndarray(input_shape, dtype, input_tensor_populator)
+
+
+@tvm.testing.fixture
+def transformed_expected_output_np(expected_output_np, output_layout):
+    return transform_numpy(expected_output_np, "nhwc", output_layout)
+
+
+@tvm.testing.fixture
+def transformed_input_np_padded(input_np_padded, input_layout):
+    return transform_numpy(input_np_padded, "nhwc", input_layout)
+
+
+class TestmaxPool2dSlice:
+    _param_descs = [
+        "out_shape",  # output_shape
+        "kernel",  # kernel
+        "stride",  # stride
+        "dil",  # dilation
+        "pad",  # padding
+        "ceil",  # ceil_mode
+        "cnt_padded",  # count_include_pad
+        "out_layout",  # output_layout
+        None,  # dtype
+        None,  # input_tensor_populator
+    ]
+
+    _multitest_params = [
+        (
+            [1, 8, 8, 32],
+            [3, 3],
+            [1, 1],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 16, 16, 32],
+            [3, 3],
+            [1, 1],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 8, 8, 32],
+            [8, 8],
+            [1, 1],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        # Test non-one stride and dilation
+        (
+            [1, 8, 8, 32],
+            [3, 3],
+            [2, 3],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 8, 8, 32],
+            [3, 3],
+            [2, 2],
+            [2, 2],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 8, 8, 32],
+            [3, 3],
+            [2, 2],
+            [2, 3],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        # Test non-zero padding
+        (
+            [1, 8, 8, 32],
+            [3, 3],
+            [1, 1],
+            [1, 1],
+            [1, 1, 1, 1],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 8, 8, 32],
+            [3, 3],
+            [1, 1],
+            [1, 1],
+            [1, 2, 3, 4],
+            False,
+            True,
+            "nhwc-8h2w32c2w-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        # Test n11c-1024c-2d layout which will require input and output to have different layout
+        (
+            [1, 1, 1, 2048],
+            [8, 8],
+            [1, 1],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "n11c-1024c-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 1, 1, 2048],
+            [6, 6],
+            [1, 1],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "n11c-1024c-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 1, 1, 2048],
+            [3, 3],
+            [2, 2],
+            [1, 1],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "n11c-1024c-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+        (
+            [1, 1, 1, 2048],
+            [4, 4],
+            [2, 2],
+            [2, 3],
+            [0, 0, 0, 0],
+            False,
+            True,
+            "n11c-1024c-2d",
+            "float16",
+            TensorContentRandom(),
+        ),
+    ]
+
+    _param_ids = get_multitest_ids(_multitest_params, _param_descs)
+
+    input_layout = tvm.testing.parameter(
+        "nhwc-8h2w32c2w-2d",
+    )
+
+    # NOTE: input_layout is always assumed to be "nhwc-8h2w32c2w-2d"
+    (
+        output_shape,
+        kernel,
+        stride,
+        dilation,
+        padding,
+        ceil_mode,
+        count_include_pad,
+        output_layout,
+        dtype,
+        input_tensor_populator,
+    ) = tvm.testing.parameters(*_multitest_params, ids=_param_ids)
+
+    @tvm.testing.fixture
+    def expected_output_np(
+        self,
+        input_np,
+        kernel,
+        stride,
+        dilation,
+        padding,
+        ceil_mode,
+        count_include_pad,
+    ):
+        pad_before = padding[:2]
+        pad_after = padding[2:]
+        ref_np = tvm.topi.testing.poolnd_python(
+            input_np,
+            kernel,
+            stride,
+            dilation,
+            pad_before,
+            pad_after,
+            "max",  # pool_type
+            count_include_pad,
+            False,  # ceil_mode,
+            layout="NHWC",
+        )
+        return ref_np
+
+    @tvm.testing.fixture
+    def input_shape(self, output_shape, kernel, padding, stride, dilation, output_layout):
+        # Input shape without any padding; 'ceil' is being ignored from calculation:
+        o_b, o_h, o_w, o_c = output_shape
+        d_h, d_w = dilation
+        s_h, s_w = stride
+        k_h, k_w = kernel
+        pad_before_h, pad_before_w = padding[:2]
+        pad_after_h, pad_after_w = padding[2:]
+
+        if output_layout == "n11c-1024c-2d":
+            assert (
+                pad_before_w == 0 and pad_after_w == 0 and pad_before_h == 0 and pad_after_h == 0
+            ), "Padding must be zero for n11c-1024c-2d layout"
+            assert o_h == 1 and o_w == 1, "Output height and width must be 1"
+
+        in_h = (o_h - 1) * s_h + d_h * (k_h - 1) + 1 - pad_before_h - pad_after_h
+        in_w = (o_w - 1) * s_w + d_w * (k_w - 1) + 1 - pad_before_w - pad_after_w
+
+        return [o_b, in_h, in_w, o_c]
+
+    @tvm.testing.fixture
+    def input_shape_padded(self, input_shape, padding, output_layout):
+        # Input shape is adjusted to account for 'padding'. Also, due to the physical
+        # layout of the buffer, height and width are adjusted so that they are a
+        # multiple of 8 and 4 respectively.
+        # NOTE: Input layout is always assumed to be nhwc-8h2w32c2w-2d.
+        pad_before_h, pad_before_w = padding[:2]
+        pad_after_h, pad_after_w = padding[2:]
+        padded_input_height = ((input_shape[1] + pad_before_h + pad_after_h + 7) // 8) * 8
+        padded_input_width = ((input_shape[2] + pad_before_w + pad_after_w + 3) // 4) * 4
+        return [input_shape[0], padded_input_height, padded_input_width, input_shape[3]]
+
+    @tvm.testing.fixture
+    def input_np_padded(self, input_np, input_shape, input_shape_padded, padding):
+        pad_before_h, pad_before_w = padding[:2]
+        pad_after_h = input_shape_padded[1] - input_shape[1] - pad_before_h
+        pad_after_w = input_shape_padded[2] - input_shape[2] - pad_before_w
+
+        input_padded = np.pad(
+            input_np,
+            ((0, 0), (pad_before_h, pad_after_h), (pad_before_w, pad_after_w), (0, 0)),
+            "constant",
+        )
+        return input_padded
+
+    @tvm.testing.requires_hexagon
+    def test_max_pool2d_slice(
+        self,
+        stride,
+        kernel,
+        dtype,
+        dilation,
+        padding,
+        ceil_mode,  # only needed for manually obtaining the test id string
+        input_tensor_populator,  # only needed for manually obtaining the test id string
+        count_include_pad,
+        input_layout,
+        output_layout,
+        output_shape,
+        input_shape,
+        input_shape_padded,
+        input_np,
+        input_np_padded,
+        transformed_input_np_padded,
+        transformed_expected_output_np,
+        expected_output_np,
+        hexagon_session: Session,
+    ):
+        if hexagon_session._launcher._serial_number != "simulator":
+            pytest.skip(msg="Due to https://github.com/apache/tvm/issues/11928")
+
+        target_hexagon = tvm.target.hexagon("v69")
+        A = te.placeholder(input_shape_padded, name="A", dtype=dtype)
+
+        M = sl.max_pool2d_compute(A, output_shape, kernel, stride, dilation)
+
+        # tir schedule
+        tir_schedule = sl.max_pool2d_STIR_schedule(M, A, output_layout, input_layout)
+        sch = tir_schedule.mod
+
+        input_axis_separator = [4]
+        if output_layout == "nhwc-8h2w32c2w-2d":
+            output_axis_separator = [4]
+        elif output_layout == "n11c-1024c-2d":
+            output_axis_separator = [4]
+        else:
+            raise RuntimeError(f"Unexpected layout '{output_layout}'")
+
+        with tvm.transform.PassContext(opt_level=3):
+            func = tvm.build(
+                sch,
+                [A, M],
+                tvm.target.Target(target_hexagon, host=target_hexagon),
+                name="max_pool2d",
+            )
+
+        input_arr = allocate_hexagon_array(
+            hexagon_session.device,
+            data=transformed_input_np_padded,
+            axis_separators=input_axis_separator,
+            mem_scope="global.vtcm",
+        )
+        output_arr = allocate_hexagon_array(
+            hexagon_session.device,
+            transformed_expected_output_np.shape,
+            dtype,
+            axis_separators=output_axis_separator,
+            mem_scope="global.vtcm",
+        )
+
+        current_test_id = get_test_id(

Review Comment:
   What's "current_test_id" used for?



##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op
+
+Please note the following assumptions made by the implementation:
+
+1) The input must be padded in advance to account for 'padding'. In addition,
+   both input and output must be padded as per the physical buffer layout.
+2) The current implementation assumes 'count_include_pad' to be 'True'. It can be
+   modified to support 'False' case but the element count for the pooling window
+   must be pre-computed and provided as an input to reduce the run-time overhead.
+3) 'padding' is ignored. It must be handled outside of the sliced op.
+4) Please note that this implementation will not work if the output includes any
+   physical layout related padding as it can result into out-of-bound access
+   for the input.
+"""
+
+from tvm import te
+from tvm import tir
+from ..utils import get_layout_transform_fn
+
+import io
+import sys
+from typing import *
+
+
+def validate_out_shape(out_shape, in_shape, kernel, stride, dilation):
+    """Validate output shape"""
+    _, oh, ow, _ = out_shape
+    _, ih, iw, _ = in_shape
+    kh, kw = kernel
+    sh, sw = stride
+    dh, dw = dilation
+    if ih < (oh - 1) * sh + dh * (kh - 1) + 1:
+        raise RuntimeError("Output height is too large")
+    if iw < (ow - 1) * sw + dw * (kw - 1) + 1:
+        raise RuntimeError("Output width is too large")
+
+
+def max_pool2d_compute(A, out_shape, kernel, stride, dilation):
+    """max_pool2d compute"""
+    kh, kw = kernel
+    rh = te.reduce_axis((0, kh), name="rh")
+    rw = te.reduce_axis((0, kw), name="rw")
+    ob, oh, ow, oc = out_shape
+    if isinstance(ob, int):
+        validate_out_shape(out_shape, A.shape, kernel, stride, dilation)
+
+    sh, sw = stride
+    dh, dw = dilation
+
+    Max = te.compute(
+        out_shape,
+        lambda b, h, w, c: te.max(
+            A[b, h * sh + dh * rh, w * sw + dw * rw, c].astype(A.dtype), axis=[rh, rw]
+        ),
+        name="max",
+    )
+    return Max
+
+
+def STIR_schedule_nhwc_8h2w32c2w(outs, ins, output_layout: str, input_layout: str):
+    """Schedule for input and output layout nhwc-8h2w32c2w"""
+    func = te.create_prim_func([ins, outs])
+    s = tir.Schedule(func)
+
+    # NOTE!!! This scheduling logic is a work in progress.
+    # It is not known to ultimately result in near-optimal Hexagon performance.
+    # The schedule below strives to implement these heuristics:
+    #
+    # - Each 2048-byte chunk of the output tensor should be visited only once, if possible.
+    #
+    # - The resulting object code should use Hexagon v69's HVX SIMD units if at all possible.
+    #   (The HVX SIMD registers are 2048 bytes long, so each "chunk" of the output tensor
+    #   fits exactly in a single HVX SIMD register.)
+
+    Max = s.get_block("max")
+
+    input_transform_fn = get_layout_transform_fn(input_layout)
+    output_transform_fn = get_layout_transform_fn(output_layout)
+
+    s.transform_layout(Max, ("read", 0), input_transform_fn)
+    s.transform_layout(Max, ("write", 0), output_transform_fn)
+
+    # Restructure the loop nestings to have this overall structure:
+    # (loop over different 2048-byte output-tensor chunks) : n, ho, wo, co   }- the first level of a two-level tensor layout
+    #    (loop within one 2048-byte output-tensor chunk) : hi, wio, ci, wii  }- the second level of a two-level tensor layout
+    #        (loop over reduction axes) : rh, rw                             }- loop over multiple elements of the input tensor
+    #
+    # Note: This schedule is a work in progress.  We *expect* that it's
+    # crucially important for the loops to have this relative ordering:
+    #    n ... ho ... wo ... co ... hi ... wio ... ci ... wii
+    # because it lets us visit each of the 2048-byte output chunks precisely once.
+
+    (
+        n,
+        h,
+        w,
+        c,
+        rh,
+        rw,
+    ) = s.get_loops(Max)
+
+    # Restructure the loops from NHWC to nhwc_8h2w32c2w, with loops for 'sum's reduction

Review Comment:
   Did you mean "Max's reduction" in the comment?



-- 
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] jverma-quic commented on a diff in pull request #12169: [hexagon][topi] add sliced max_pool2

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


##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op
+
+Please note the following assumptions made by the implementation:
+
+1) The input must be padded in advance to account for 'padding'. In addition,
+   both input and output must be padded as per the physical buffer layout.
+2) The current implementation assumes 'count_include_pad' to be 'True'. It can be
+   modified to support 'False' case but the element count for the pooling window
+   must be pre-computed and provided as an input to reduce the run-time overhead.
+3) 'padding' is ignored. It must be handled outside of the sliced op.
+4) Please note that this implementation will not work if the output includes any
+   physical layout related padding as it can result into out-of-bound access
+   for the input.
+"""
+
+from tvm import te
+from tvm import tir
+from ..utils import get_layout_transform_fn
+
+import io
+import sys
+from typing import *
+
+
+def validate_out_shape(out_shape, in_shape, kernel, stride, dilation):
+    """Validate output shape"""
+    _, oh, ow, _ = out_shape
+    _, ih, iw, _ = in_shape
+    kh, kw = kernel
+    sh, sw = stride
+    dh, dw = dilation
+    if ih < (oh - 1) * sh + dh * (kh - 1) + 1:
+        raise RuntimeError("Output height is too large")
+    if iw < (ow - 1) * sw + dw * (kw - 1) + 1:
+        raise RuntimeError("Output width is too large")
+
+
+def max_pool2d_compute(A, out_shape, kernel, stride, dilation):
+    """max_pool2d compute"""
+    kh, kw = kernel
+    rh = te.reduce_axis((0, kh), name="rh")
+    rw = te.reduce_axis((0, kw), name="rw")
+    ob, oh, ow, oc = out_shape
+    if isinstance(ob, int):
+        validate_out_shape(out_shape, A.shape, kernel, stride, dilation)
+
+    sh, sw = stride
+    dh, dw = dilation
+
+    Max = te.compute(
+        out_shape,
+        lambda b, h, w, c: te.max(
+            A[b, h * sh + dh * rh, w * sw + dw * rw, c].astype(A.dtype), axis=[rh, rw]
+        ),
+        name="max",
+    )
+    return Max
+
+
+def STIR_schedule_nhwc_8h2w32c2w(outs, ins, output_layout: str, input_layout: str):
+    """Schedule for input and output layout nhwc-8h2w32c2w"""
+    func = te.create_prim_func([ins, outs])
+    s = tir.Schedule(func)
+
+    # NOTE!!! This scheduling logic is a work in progress.
+    # It is not known to ultimately result in near-optimal Hexagon performance.
+    # The schedule below strives to implement these heuristics:
+    #
+    # - Each 2048-byte chunk of the output tensor should be visited only once, if possible.
+    #
+    # - The resulting object code should use Hexagon v69's HVX SIMD units if at all possible.
+    #   (The HVX SIMD registers are 2048 bytes long, so each "chunk" of the output tensor
+    #   fits exactly in a single HVX SIMD register.)
+
+    Max = s.get_block("max")
+
+    input_transform_fn = get_layout_transform_fn(input_layout)
+    output_transform_fn = get_layout_transform_fn(output_layout)
+
+    s.transform_layout(Max, ("read", 0), input_transform_fn)
+    s.transform_layout(Max, ("write", 0), output_transform_fn)
+
+    # Restructure the loop nestings to have this overall structure:
+    # (loop over different 2048-byte output-tensor chunks) : n, ho, wo, co   }- the first level of a two-level tensor layout
+    #    (loop within one 2048-byte output-tensor chunk) : hi, wio, ci, wii  }- the second level of a two-level tensor layout
+    #        (loop over reduction axes) : rh, rw                             }- loop over multiple elements of the input tensor
+    #
+    # Note: This schedule is a work in progress.  We *expect* that it's
+    # crucially important for the loops to have this relative ordering:
+    #    n ... ho ... wo ... co ... hi ... wio ... ci ... wii
+    # because it lets us visit each of the 2048-byte output chunks precisely once.
+
+    (
+        n,
+        h,
+        w,
+        c,
+        rh,
+        rw,
+    ) = s.get_loops(Max)
+
+    # Restructure the loops from NHWC to nhwc_8h2w32c2w, with loops for 'sum's reduction
+    # axes at the very end.
+    ho, hi = s.split(h, [None, 8])
+    wo, wi = s.split(w, [None, 4])
+    wio, wii = s.split(wi, [None, 2])
+    co, ci = s.split(c, [None, 32])
+    s.reorder(n, ho, wo, co, hi, wio, ci, wii, rh, rw)

Review Comment:
   Yes, I have been wondering about that too. I know that transform_layout on a te.compute does the splitting as per the layout but this feature was missing from stir transform_layout and doesn't look like it has been added yet. Otherwise, it wouldn't caused failures. 



-- 
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 pull request #12169: [hexagon][topi] add sliced max_pool2

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

   CC: @jverma-quic @csullivan @Lunderberg 


-- 
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 #12169: [hexagon][topi] add sliced max_pool2

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


-- 
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 #12169: [hexagon][topi] add sliced max_pool2

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


##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op
+
+Please note the following assumptions made by the implementation:
+
+1) The input must be padded in advance to account for 'padding'. In addition,
+   both input and output must be padded as per the physical buffer layout.
+2) The current implementation assumes 'count_include_pad' to be 'True'. It can be
+   modified to support 'False' case but the element count for the pooling window
+   must be pre-computed and provided as an input to reduce the run-time overhead.
+3) 'padding' is ignored. It must be handled outside of the sliced op.
+4) Please note that this implementation will not work if the output includes any
+   physical layout related padding as it can result into out-of-bound access
+   for the input.
+"""
+
+from tvm import te
+from tvm import tir
+from ..utils import get_layout_transform_fn
+
+import io
+import sys
+from typing import *
+
+
+def validate_out_shape(out_shape, in_shape, kernel, stride, dilation):
+    """Validate output shape"""
+    _, oh, ow, _ = out_shape
+    _, ih, iw, _ = in_shape
+    kh, kw = kernel
+    sh, sw = stride
+    dh, dw = dilation
+    if ih < (oh - 1) * sh + dh * (kh - 1) + 1:
+        raise RuntimeError("Output height is too large")
+    if iw < (ow - 1) * sw + dw * (kw - 1) + 1:
+        raise RuntimeError("Output width is too large")
+
+
+def max_pool2d_compute(A, out_shape, kernel, stride, dilation):
+    """max_pool2d compute"""
+    kh, kw = kernel
+    rh = te.reduce_axis((0, kh), name="rh")
+    rw = te.reduce_axis((0, kw), name="rw")
+    ob, oh, ow, oc = out_shape
+    if isinstance(ob, int):
+        validate_out_shape(out_shape, A.shape, kernel, stride, dilation)
+
+    sh, sw = stride
+    dh, dw = dilation
+
+    Max = te.compute(
+        out_shape,
+        lambda b, h, w, c: te.max(
+            A[b, h * sh + dh * rh, w * sw + dw * rw, c].astype(A.dtype), axis=[rh, rw]
+        ),
+        name="max",
+    )
+    return Max
+
+
+def STIR_schedule_nhwc_8h2w32c2w(outs, ins, output_layout: str, input_layout: str):
+    """Schedule for input and output layout nhwc-8h2w32c2w"""
+    func = te.create_prim_func([ins, outs])
+    s = tir.Schedule(func)
+
+    # NOTE!!! This scheduling logic is a work in progress.
+    # It is not known to ultimately result in near-optimal Hexagon performance.
+    # The schedule below strives to implement these heuristics:
+    #
+    # - Each 2048-byte chunk of the output tensor should be visited only once, if possible.
+    #
+    # - The resulting object code should use Hexagon v69's HVX SIMD units if at all possible.
+    #   (The HVX SIMD registers are 2048 bytes long, so each "chunk" of the output tensor
+    #   fits exactly in a single HVX SIMD register.)
+
+    Max = s.get_block("max")
+
+    input_transform_fn = get_layout_transform_fn(input_layout)
+    output_transform_fn = get_layout_transform_fn(output_layout)
+
+    s.transform_layout(Max, ("read", 0), input_transform_fn)
+    s.transform_layout(Max, ("write", 0), output_transform_fn)
+
+    # Restructure the loop nestings to have this overall structure:
+    # (loop over different 2048-byte output-tensor chunks) : n, ho, wo, co   }- the first level of a two-level tensor layout
+    #    (loop within one 2048-byte output-tensor chunk) : hi, wio, ci, wii  }- the second level of a two-level tensor layout
+    #        (loop over reduction axes) : rh, rw                             }- loop over multiple elements of the input tensor
+    #
+    # Note: This schedule is a work in progress.  We *expect* that it's
+    # crucially important for the loops to have this relative ordering:
+    #    n ... ho ... wo ... co ... hi ... wio ... ci ... wii
+    # because it lets us visit each of the 2048-byte output chunks precisely once.
+
+    (
+        n,
+        h,
+        w,
+        c,
+        rh,
+        rw,
+    ) = s.get_loops(Max)
+
+    # Restructure the loops from NHWC to nhwc_8h2w32c2w, with loops for 'sum's reduction
+    # axes at the very end.
+    ho, hi = s.split(h, [None, 8])
+    wo, wi = s.split(w, [None, 4])
+    wio, wii = s.split(wi, [None, 2])
+    co, ci = s.split(c, [None, 32])
+    s.reorder(n, ho, wo, co, hi, wio, ci, wii, rh, rw)

Review Comment:
   From talking with @Lunderberg , I got the impression that some or all of these loop manipulations were expected to already be done by the earlier call to `transform_layout`.  
   
   TODO(cconvey): Figure out if my understanding was wrong, or there's a bug in transform_layout.
   
   EDIT: Just remembered that this code is dealing with S-TIR scheduling, not TE scheduling.  I.e., we call `tir.schedule(...)` rather than `te.create_schedule(...)`.
   
   We currently expect the automatic loop reordering to happen when we call `layout_transform` on TE schedules, but to _not_ happen when we call `layout_transform` on _S-TIR_ schedules.  So this code is actually working as expected.



-- 
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] jverma-quic commented on a diff in pull request #12169: [hexagon][topi] add sliced max_pool2

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


##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op
+
+Please note the following assumptions made by the implementation:
+
+1) The input must be padded in advance to account for 'padding'. In addition,
+   both input and output must be padded as per the physical buffer layout.
+2) The current implementation assumes 'count_include_pad' to be 'True'. It can be
+   modified to support 'False' case but the element count for the pooling window
+   must be pre-computed and provided as an input to reduce the run-time overhead.
+3) 'padding' is ignored. It must be handled outside of the sliced op.
+4) Please note that this implementation will not work if the output includes any
+   physical layout related padding as it can result into out-of-bound access
+   for the input.
+"""
+
+from tvm import te
+from tvm import tir
+from ..utils import get_layout_transform_fn
+
+import io
+import sys
+from typing import *
+
+
+def validate_out_shape(out_shape, in_shape, kernel, stride, dilation):
+    """Validate output shape"""
+    _, oh, ow, _ = out_shape
+    _, ih, iw, _ = in_shape
+    kh, kw = kernel
+    sh, sw = stride
+    dh, dw = dilation
+    if ih < (oh - 1) * sh + dh * (kh - 1) + 1:
+        raise RuntimeError("Output height is too large")
+    if iw < (ow - 1) * sw + dw * (kw - 1) + 1:
+        raise RuntimeError("Output width is too large")
+
+
+def max_pool2d_compute(A, out_shape, kernel, stride, dilation):
+    """max_pool2d compute"""
+    kh, kw = kernel
+    rh = te.reduce_axis((0, kh), name="rh")
+    rw = te.reduce_axis((0, kw), name="rw")
+    ob, oh, ow, oc = out_shape
+    if isinstance(ob, int):
+        validate_out_shape(out_shape, A.shape, kernel, stride, dilation)
+
+    sh, sw = stride
+    dh, dw = dilation
+
+    Max = te.compute(
+        out_shape,
+        lambda b, h, w, c: te.max(
+            A[b, h * sh + dh * rh, w * sw + dw * rw, c].astype(A.dtype), axis=[rh, rw]
+        ),
+        name="max",
+    )
+    return Max
+
+
+def STIR_schedule_nhwc_8h2w32c2w(outs, ins, output_layout: str, input_layout: str):
+    """Schedule for input and output layout nhwc-8h2w32c2w"""
+    func = te.create_prim_func([ins, outs])
+    s = tir.Schedule(func)
+
+    # NOTE!!! This scheduling logic is a work in progress.
+    # It is not known to ultimately result in near-optimal Hexagon performance.
+    # The schedule below strives to implement these heuristics:
+    #
+    # - Each 2048-byte chunk of the output tensor should be visited only once, if possible.
+    #
+    # - The resulting object code should use Hexagon v69's HVX SIMD units if at all possible.
+    #   (The HVX SIMD registers are 2048 bytes long, so each "chunk" of the output tensor
+    #   fits exactly in a single HVX SIMD register.)
+
+    Max = s.get_block("max")
+
+    input_transform_fn = get_layout_transform_fn(input_layout)
+    output_transform_fn = get_layout_transform_fn(output_layout)
+
+    s.transform_layout(Max, ("read", 0), input_transform_fn)
+    s.transform_layout(Max, ("write", 0), output_transform_fn)
+
+    # Restructure the loop nestings to have this overall structure:
+    # (loop over different 2048-byte output-tensor chunks) : n, ho, wo, co   }- the first level of a two-level tensor layout
+    #    (loop within one 2048-byte output-tensor chunk) : hi, wio, ci, wii  }- the second level of a two-level tensor layout
+    #        (loop over reduction axes) : rh, rw                             }- loop over multiple elements of the input tensor
+    #
+    # Note: This schedule is a work in progress.  We *expect* that it's
+    # crucially important for the loops to have this relative ordering:
+    #    n ... ho ... wo ... co ... hi ... wio ... ci ... wii
+    # because it lets us visit each of the 2048-byte output chunks precisely once.
+
+    (
+        n,
+        h,
+        w,
+        c,
+        rh,
+        rw,
+    ) = s.get_loops(Max)
+
+    # Restructure the loops from NHWC to nhwc_8h2w32c2w, with loops for 'sum's reduction
+    # axes at the very end.
+    ho, hi = s.split(h, [None, 8])
+    wo, wi = s.split(w, [None, 4])
+    wio, wii = s.split(wi, [None, 2])
+    co, ci = s.split(c, [None, 32])
+    s.reorder(n, ho, wo, co, hi, wio, ci, wii, rh, rw)
+
+    # TODO: Enable vectorization.
+    # Hexagon v69's HVX units support SIMD operations on 128-element float16 vectors.

Review Comment:
   128-element float16 -> 64-element float16



##########
tests/python/contrib/test_hexagon/topi/test_max_pool2d_slice.py:
##########
@@ -0,0 +1,407 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+import numpy as np
+from typing import *
+
+from tvm import te
+import tvm.testing
+from tvm.topi import testing
+from tvm.contrib.hexagon.build import HexagonLauncher
+from tvm.contrib.hexagon.session import Session
+import tvm.topi.hexagon.slice_ops as sl
+from ..infrastructure import allocate_hexagon_array, transform_numpy
+from ..pytest_util import (
+    get_numpy_dtype_info,
+    get_test_id,
+    get_multitest_ids,
+    create_populated_numpy_ndarray,
+    TensorContentConstant,
+    TensorContentRandom,
+    TensorContentDtypeMin,
+    TensorContentDtypeMax,
+    TensorContentSequentialCOrder,

Review Comment:
   Do we need to import all of these? I only see TensorContentRandom being used. 



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

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 #12169: [hexagon][topi] add sliced max_pool2

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


##########
python/tvm/topi/hexagon/slice_ops/max_pool2d.py:
##########
@@ -0,0 +1,193 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-variable, unused-argument, too-many-locals
+
+""" Compute and schedule for avg_pool2d slice op
+
+Please note the following assumptions made by the implementation:
+
+1) The input must be padded in advance to account for 'padding'. In addition,
+   both input and output must be padded as per the physical buffer layout.
+2) The current implementation assumes 'count_include_pad' to be 'True'. It can be
+   modified to support 'False' case but the element count for the pooling window
+   must be pre-computed and provided as an input to reduce the run-time overhead.
+3) 'padding' is ignored. It must be handled outside of the sliced op.
+4) Please note that this implementation will not work if the output includes any
+   physical layout related padding as it can result into out-of-bound access
+   for the input.
+"""
+
+from tvm import te
+from tvm import tir
+from ..utils import get_layout_transform_fn
+
+import io
+import sys
+from typing import *
+
+
+def validate_out_shape(out_shape, in_shape, kernel, stride, dilation):
+    """Validate output shape"""
+    _, oh, ow, _ = out_shape
+    _, ih, iw, _ = in_shape
+    kh, kw = kernel
+    sh, sw = stride
+    dh, dw = dilation
+    if ih < (oh - 1) * sh + dh * (kh - 1) + 1:
+        raise RuntimeError("Output height is too large")
+    if iw < (ow - 1) * sw + dw * (kw - 1) + 1:
+        raise RuntimeError("Output width is too large")
+
+
+def max_pool2d_compute(A, out_shape, kernel, stride, dilation):
+    """max_pool2d compute"""
+    kh, kw = kernel
+    rh = te.reduce_axis((0, kh), name="rh")
+    rw = te.reduce_axis((0, kw), name="rw")
+    ob, oh, ow, oc = out_shape
+    if isinstance(ob, int):
+        validate_out_shape(out_shape, A.shape, kernel, stride, dilation)
+
+    sh, sw = stride
+    dh, dw = dilation
+
+    Max = te.compute(
+        out_shape,
+        lambda b, h, w, c: te.max(
+            A[b, h * sh + dh * rh, w * sw + dw * rw, c].astype(A.dtype), axis=[rh, rw]
+        ),
+        name="max",
+    )
+    return Max
+
+
+def STIR_schedule_nhwc_8h2w32c2w(outs, ins, output_layout: str, input_layout: str):
+    """Schedule for input and output layout nhwc-8h2w32c2w"""
+    func = te.create_prim_func([ins, outs])
+    s = tir.Schedule(func)
+
+    # NOTE!!! This scheduling logic is a work in progress.
+    # It is not known to ultimately result in near-optimal Hexagon performance.
+    # The schedule below strives to implement these heuristics:
+    #
+    # - Each 2048-byte chunk of the output tensor should be visited only once, if possible.

Review Comment:
   When I wrote this, I somehow had the wrong size of HVX SIMD vectors in my head.  Their actual size is 128 bytes.  I need to update (at least) this file's comments accordingly,



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