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/04/28 12:01:40 UTC

[GitHub] [tvm] elvin-n opened a new pull request, #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

elvin-n opened a new pull request, #11161:
URL: https://github.com/apache/tvm/pull/11161

   - There are 5 compute/schedules: conv2d for NCHW/NHWC, depthwise_conv2d
     for NCHW/NHWC, average pooling
   - Fix of dynamically allocated textures caching
   - Add texture-nhwc scope
   - Fix issue with codegen of vars having non acceptable symbols


-- 
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] elvin-n commented on pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on PR #11161:
URL: https://github.com/apache/tvm/pull/11161#issuecomment-1117947459

   > Note as this is a squash I would suggest use of `Co-authored-by` in the commit to reflect the co-authorship.
   
   Done


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

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

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


[GitHub] [tvm] elvin-n commented on pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on PR #11161:
URL: https://github.com/apache/tvm/pull/11161#issuecomment-1115334261

   @csullivan Could you please take a look?


-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):

Review Comment:
   Instead find the smallest divisor and calculate the largest divisor as the division of `value` by the smallest divisor.



-- 
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] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865391232


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )

Review Comment:
   added comment and reference to rfc



-- 
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] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865390876


##########
python/tvm/topi/adreno/conv2d_alter_op.py:
##########
@@ -0,0 +1,211 @@
+# 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,no-member
+"""Conv2D alter op and legalize functions for x86"""

Review Comment:
   done



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):

Review Comment:
   done



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4

Review Comment:
   done



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865391074


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail

Review Comment:
   tried to do my best



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
src/runtime/opencl/opencl_common.h:
##########
@@ -345,6 +345,7 @@ struct BufferDescriptor {
      *         e.g. image2d[height=O, width=IHW]
      */
     kImage2DWeight,
+    kTexture2DNHWC,

Review Comment:
   We can now support arbitrary layouts with `transform_layout` which I will suggest we move to. It will require some rework on the TIR lowering. I don't suggest this block these schedules from being upstreamed now, but we should circle back on this soon. 



##########
python/tvm/relay/op/strategy/adreno.py:
##########
@@ -0,0 +1,162 @@
+# 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.
+"""Definition of adreno operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from tvm import topi
+from .generic import *
+from .. import op as _op
+
+
+@conv2d_NCHWc_strategy.register("adreno")
+@conv2d_strategy.register("adreno")
+def conv2d_strategy_adreno(attrs, inputs, out_type, target):
+    """conv2d adreno strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation_h, dilation_w = attrs.get_int_tuple("dilation")
+    groups = attrs.groups
+    data_layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if (data_layout == "NCHW" and kernel_layout == "OIHW") or (
+            data_layout == "NCHW4c" and kernel_layout == "OIHW4o"
+        ):
+            if out_type.dtype == "float16":
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.adreno.conv2d_nchwc),
+                    wrap_topi_schedule(topi.adreno.schedule_conv2d_nchwc),
+                    name="conv2d_nchwc.image2d",
+                    plevel=10,
+                )
+            strategy.add_implementation(
+                wrap_compute_conv2d(topi.adreno.conv2d_nchwc_acc32),
+                wrap_topi_schedule(topi.adreno.schedule_conv2d_nchwc_acc32),
+                name="conv2d_nchwc_tpack.image2d",
+                plevel=20,
+            )
+        elif (data_layout == "NHWC" and kernel_layout == "HWIO") or (
+            data_layout == "NHWC4c" and kernel_layout == "HWIO4o"
+        ):
+            if out_type.dtype == "float16":
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.adreno.conv2d_nhwc),
+                    wrap_topi_schedule(topi.adreno.schedule_conv2d_nhwc),
+                    name="conv2d_nhwc.image2d",
+                    plevel=10,
+                )
+            strategy.add_implementation(
+                wrap_compute_conv2d(topi.adreno.conv2d_nhwc_acc32),
+                wrap_topi_schedule(topi.adreno.schedule_conv2d_nhwc_acc32),
+                name="conv2d_nhwc_acc32.image2d",
+                plevel=20,
+            )
+        else:
+            raise RuntimeError(
+                "Layout not supported: ("
+                + data_layout
+                + ", "
+                + kernel_layout
+                + ") - only support NCHW4c / OIHW4o and NHWC / HWOI layouts for conv2d"
+            )
+    else:
+        # cannot use is_depthwise_conv2d because it does not know about NHWC4c/HWOI4o layouts
+        if data_layout == "NCHW":
+            ic = data.shape[1]
+        elif data_layout == "NCHW4c":
+            ic = data.shape[1] * data.shape[4]
+        elif data_layout == "NHWC":
+            ic = data.shape[3]
+        elif data_layout == "NHWC4c":
+            ic = data.shape[3] * data.shape[4]
+        else:
+            # TODO(amalyshe) add proper error raising

Review Comment:
   Address the TODOs



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail

Review Comment:
   nit: consider referring to this as `padding_tail` so that it's clear this isn't the remainder of a floordiv. anything to make this a little more clear upfront, took me a bit to understand given the current naming convention. Same comment for filter api below.
   



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):

Review Comment:
   snake_case to match the rest of the file



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )
+
+    # compute:
+    if layout == "NCHW":
+        reordered_data = te.compute(
+            [batch, in_channel_chunks, in_height, in_width, in_channel_block],
+            _reorder_data_nchw,
+            name="input_pack",
+            tag="input_pack",
+        )
+    elif layout == "NHWC":
+        reordered_data = te.compute(
+            [batch, in_height, in_width, in_channel_chunks, in_channel_block],
+            _reorder_data_nhwc,
+            name="input_pack",
+            tag="input_pack",
+        )
+    else:
+        assert False, "Adreno util function pack_input does not accept unknown layout"
+    return reordered_data
+
+
+def pack_filter(
+    Filter,
+    layout,
+    out_channel_chunks,
+    out_channel_block,
+    out_channel_tail,
+    in_filter_channels,
+    in_data_channel_chunks,
+    in_data_channel_block,
+    in_data_channel_tail,
+    kernel_h,
+    kernel_w,
+):
+    """
+    Adds compute stages for packing of the filter in runtime. Extends channels dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Filter: tvm.te.Tensor
+        Filter tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    out_channel_chunks: int
+        Number of chunks for filters
+
+    out_channel_block: int
+        Size of the block
+
+    out_channel_tail: int
+        Original size of the latest chunk of output filters
+
+    in_filter_channels: int
+        Number of filter channels. might be different vs input channels in the
+        data due to groups/depthwise nature
+
+    in_data_channel_chunks: int
+        Number of chunks by channels for input data
+
+    in_data_channel_block: int
+        Size of the block for input data channels
+
+    in_data_channel_tail
+        Original size of the latest chunk for input data channels
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+    """
+    pad_value = tvm.tir.const(0, Filter.dtype)
+
+    def _reorder_weights_depthwise_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_depthwise_hwoi(*indices):
+        conditionA = []
+        conditionA.append(indices[2] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2] * out_channel_block + indices[4], indices[3]],
+        )
+
+    def _reorder_weights_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[1] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_hwio(*indices):
+        conditionA = []
+        conditionA.append(indices[3] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[2] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2], indices[3] * out_channel_block + indices[4]],
+        )
+
+    if in_filter_channels == 1:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_depthwise_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWOI":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, out_channel_chunks, in_filter_channels, out_channel_block],
+                _reorder_weights_depthwise_hwoi,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    else:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWIO":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block],
+                _reorder_weights_hwio,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    return reordered_filter
+
+
+def expand_spatial_dimensions(
+    in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+):
+    """
+    Expands spatial dimensions to be dividable by factor 4. This will allow us to do extrimely
+    better parallel computation on GPU. The drawback of this solution - it will be number of
+    useless computations. By fact the speed-up of parallelism significantly overcomes the slowdown
+    of extra compute and eventuially this is useful approach, at least for GPU
+
+    Parameters
+    ----------
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the featrue map
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+
+    dilation_h: int
+        Vertical dilation of the conv2d kernel
+
+    dilation_w: int
+        Horizontal dilation of the conv2d kernel
+
+    padding: tuple or list
+        Conv2d paddings
+
+    stride_h: int
+        Vertical stride  of the conv2d kernel
+
+    stride_w: int
+        Horizontal stride  of the conv2d kernel
+    """
+    dilated_kernel_h = (kernel_h - 1) * dilation_h + 1
+    dilated_kernel_w = (kernel_w - 1) * dilation_w + 1
+
+    pad_top, pad_left, pad_down, pad_right = nn.get_pad_tuple(
+        padding, (dilated_kernel_h, dilated_kernel_w)
+    )
+
+    out_height_orig = out_height = simplify(
+        (in_height - dilated_kernel_h + pad_top + pad_down) // stride_h + 1
+    )
+    out_width_orig = out_width = simplify(
+        (in_width - dilated_kernel_w + pad_left + pad_right) // stride_w + 1
+    )
+
+    # can output shape be divded by 2 or even 4?
+    # if it cannot be divided, need to extend for further help with split
+    # theortically there should be addition padding for inputs, but it will be optimized by
+    # cache_read InferBound. We must proceed pad here exactly to produce tensor which is
+    # required for calculation of original out size, not more! In other case intermediate
+    # tensor might be allcoated with less sizes while compute will try to fill the expanded
+    # one - data discrepancy as a result
+    # And in case of textures it is not a problem if we provide texture of less size because
+    # 1. It is not important which values would be for extra calc - these calculations are
+    #    required only for better utilizatin of GPU fit to working groups
+    # 2. When we request pixel out opf bound, texture will handle this correctly. As mentioned
+    #    above, the value itself is not important
+    if out_height % 2 != 0:
+        out_height += 1
+    if out_width % 2 != 0:
+        out_width += 1
+
+    if out_height % 4 != 0:
+        out_height += 2
+    if out_width % 4 != 0:
+        out_width += 2
+    return out_height_orig, out_height, out_width_orig, out_width
+
+
+def add_pad(
+    data,
+    layout,
+    out_height,
+    out_width,
+    kernel_h,
+    kernel_w,
+    dilation_h,
+    dilation_w,
+    padding,
+    stride_h,
+    stride_w,
+):
+    """Computes required padding values by the parameters of conv2d and adds
+        compute for extending of original tensor
+
+    Parameters
+    ----------
+    data: tvm.te.Tensor
+        5d tensor, the layout of spatial dimensions are defined as separate argument
+
+    layout: string
+        Layout of origin 4d tensor
+
+    out_height: int
+        Height of the output feature map
+
+    out_width: int
+        Width of the output feature map
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+
+    dilation_h: int
+        Height dilation value from conv2d attributes
+
+    dilation_w: int
+        Width dilation value from conv2d attributes
+
+    padding: list / tuple of n ints
+        Padding values from conv2d attributes
+
+    stride_h: int
+        Height stride value from conv2d attributes
+
+    stride_w: int
+        Width stride value from conv2d attributes
+
+    Returns
+    -------
+    Output : tvm.te.Tensor
+        n-D, the same layout as Input.
+    """
+    dilated_kernel_h = (kernel_h - 1) * dilation_h + 1
+    dilated_kernel_w = (kernel_w - 1) * dilation_w + 1
+    pad_top, pad_left, pad_down, pad_right = nn.get_pad_tuple(
+        padding, (dilated_kernel_h, dilated_kernel_w)
+    )
+
+    # compute graph
+    if layout == "NCHW":
+        y_axis = 2
+        x_axis = 3
+        if len(data.shape) == 4:
+            _, _, in_height, in_width = data.shape
+        else:
+            _, _, in_height, in_width, _ = data.shape
+    elif layout == "NHWC":
+        y_axis = 1
+        x_axis = 2
+        if len(data.shape) == 4:
+            _, in_height, in_width, _ = data.shape
+        else:
+            _, in_height, in_width, _, _ = data.shape
+    else:
+        assert False, "not supported layout in adreno util add_pad"
+    pad_before = [0, 0, 0, 0, 0]
+    pad_after = [0, 0, 0, 0, 0]
+    pad_before[y_axis] = pad_top
+    pad_before[x_axis] = pad_left
+    pad_after[y_axis] = pad_down
+    pad_after[x_axis] = pad_right
+
+    # calculation of real used input size:
+    input_latest_w = (out_width - 1) * stride_w + (kernel_w - 1) * dilation_w + 1
+    input_latest_h = (out_height - 1) * stride_h + (kernel_h - 1) * dilation_h + 1
+    if input_latest_w < in_width + pad_before[x_axis] + pad_after[x_axis]:
+        pad_after[x_axis] -= in_width + pad_before[x_axis] + pad_after[x_axis] - input_latest_w
+    if input_latest_h < in_height + pad_before[y_axis] + pad_after[y_axis]:
+        pad_after[y_axis] -= in_height + pad_before[y_axis] + pad_after[y_axis] - input_latest_h
+    return nn.pad(data, pad_before, pad_after, name="pad_temp")
+
+
+def bind_data_copy(stage, axis_to_vectorize=None):
+    """
+    Schedules the eltwise stages like copying of data or postops
+
+    Parameters
+    ----------
+    stage: tvm.te.Tensor
+
+    axis_to_vectorize:
+        Causes to split certain axis, moves inner part to the end of schedule
+        and enable vectorization by this axis
+        If parameter is not pointed, the schedule will be vectorized if the most inner
+        dim is eq to 4 (size of the vector in texture)
+    """
+    shape = get_const_tuple(stage.op.output(0).shape)
+    if axis_to_vectorize and len(shape) == 4 and shape[axis_to_vectorize] % 4 == 0:
+        ax0, ax1, ax2, ax3 = stage.op.axis
+        if axis_to_vectorize == 1:
+            oax1, iax1 = stage.split(ax1, factor=4)
+            stage.reorder(ax0, oax1, ax2, ax3, iax1)
+            stage.vectorize(iax1)
+            fused = stage.fuse(ax0, oax1, ax2, ax3)
+        elif axis_to_vectorize == 3:
+            oax3, iax3 = stage.split(ax3, factor=4)
+            stage.reorder(ax0, ax1, ax2, oax3, iax3)
+            stage.vectorize(iax3)
+            fused = stage.fuse(ax0, ax1, ax2, oax3)
+
+        ftc = numpy.prod(shape) / 4
+        div = getDiv(ftc, 128)
+        block, thread = stage.split(fused, factor=div)
+
+        stage.bind(block, te.thread_axis("blockIdx.z"))
+        stage.bind(thread, te.thread_axis("threadIdx.z"))
+    else:
+        axes = stage.op.axis
+        fused = stage.fuse(*axes[:-1])
+        if shape[-1] <= 32:
+            ftc = numpy.prod(shape[:-1])
+            div = getDiv(ftc, 64)
+            block, thread = stage.split(fused, factor=div)
+            stage.bind(block, te.thread_axis("blockIdx.x"))
+            stage.bind(thread, te.thread_axis("threadIdx.x"))
+            if shape[-1] == 4:
+                stage.vectorize(axes[-1])
+        else:
+            stage.bind(fused, te.thread_axis("blockIdx.x"))
+            stage.bind(*axes[-1:], te.thread_axis("threadIdx.x"))
+
+
+def get_texture_storage(shape):
+    """
+    Returns the texture layout acceptable for the shape
+
+    Parameters
+    ----------
+    shape: array
+        Shape of the tensor to be packed to texture
+    """
+    # certain limitation of the Qualcomm devices. Subject to be determined for certain device
+    # individually, but until we have access to remote device during compilation, we have to
+    # define it uniformly for all target devices
+    limit = 16384

Review Comment:
   Let us use the Target attributes for this, and specifically use the attribute preprocessor as is done for cuda [here](https://github.com/apache/tvm/blob/main/src/target/target_kind.cc#L295). Add image extent to the [attribute list](https://github.com/apache/tvm/blob/96d10f84afdbb1deb5db6af444f7df660c1a1b87/include/tvm/runtime/device_api.h) for the device api and use it when calling [DetectDeviceFlag](https://github.com/apache/tvm/blob/main/src/target/target_kind.cc#L114) to query the size limits of the opencl image on the remote device. 



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4

Review Comment:
   Use `block` throughout



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):

Review Comment:
   nit: `trip_count` -> `extent` or `logical_extent`



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail

Review Comment:
   Please add a comment that chunks are defined via ceildiv with these semantics:
   ```chunks = ceildiv(extent, block)```
   



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )
+
+    # compute:
+    if layout == "NCHW":
+        reordered_data = te.compute(
+            [batch, in_channel_chunks, in_height, in_width, in_channel_block],
+            _reorder_data_nchw,
+            name="input_pack",
+            tag="input_pack",
+        )
+    elif layout == "NHWC":
+        reordered_data = te.compute(
+            [batch, in_height, in_width, in_channel_chunks, in_channel_block],
+            _reorder_data_nhwc,
+            name="input_pack",
+            tag="input_pack",
+        )
+    else:
+        assert False, "Adreno util function pack_input does not accept unknown layout"
+    return reordered_data
+
+
+def pack_filter(
+    Filter,
+    layout,
+    out_channel_chunks,
+    out_channel_block,
+    out_channel_tail,
+    in_filter_channels,
+    in_data_channel_chunks,
+    in_data_channel_block,
+    in_data_channel_tail,
+    kernel_h,
+    kernel_w,
+):
+    """
+    Adds compute stages for packing of the filter in runtime. Extends channels dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Filter: tvm.te.Tensor
+        Filter tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    out_channel_chunks: int
+        Number of chunks for filters
+
+    out_channel_block: int
+        Size of the block
+
+    out_channel_tail: int
+        Original size of the latest chunk of output filters
+
+    in_filter_channels: int
+        Number of filter channels. might be different vs input channels in the
+        data due to groups/depthwise nature
+
+    in_data_channel_chunks: int
+        Number of chunks by channels for input data
+
+    in_data_channel_block: int
+        Size of the block for input data channels
+
+    in_data_channel_tail
+        Original size of the latest chunk for input data channels
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+    """
+    pad_value = tvm.tir.const(0, Filter.dtype)
+
+    def _reorder_weights_depthwise_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_depthwise_hwoi(*indices):
+        conditionA = []
+        conditionA.append(indices[2] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2] * out_channel_block + indices[4], indices[3]],
+        )
+
+    def _reorder_weights_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[1] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_hwio(*indices):
+        conditionA = []
+        conditionA.append(indices[3] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[2] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2], indices[3] * out_channel_block + indices[4]],
+        )
+
+    if in_filter_channels == 1:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_depthwise_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWOI":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, out_channel_chunks, in_filter_channels, out_channel_block],
+                _reorder_weights_depthwise_hwoi,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    else:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWIO":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block],
+                _reorder_weights_hwio,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    return reordered_filter
+
+
+def expand_spatial_dimensions(
+    in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+):
+    """
+    Expands spatial dimensions to be dividable by factor 4. This will allow us to do extrimely

Review Comment:
   Typos
   ```suggestion
       Expands spatial dimensions to be dividable by factor 4. This will allow us
   ```



##########
python/tvm/topi/adreno/conv2d_alter_op.py:
##########
@@ -0,0 +1,211 @@
+# 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,no-member
+"""Conv2D alter op and legalize functions for x86"""

Review Comment:
   Not x86



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):

Review Comment:
   Instead find the smallest divisor and calculate the largest divisor as the division of `value` by the smallest divisor.



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )
+
+    # compute:
+    if layout == "NCHW":
+        reordered_data = te.compute(
+            [batch, in_channel_chunks, in_height, in_width, in_channel_block],
+            _reorder_data_nchw,
+            name="input_pack",
+            tag="input_pack",
+        )
+    elif layout == "NHWC":
+        reordered_data = te.compute(
+            [batch, in_height, in_width, in_channel_chunks, in_channel_block],
+            _reorder_data_nhwc,
+            name="input_pack",
+            tag="input_pack",
+        )
+    else:
+        assert False, "Adreno util function pack_input does not accept unknown layout"
+    return reordered_data
+
+
+def pack_filter(
+    Filter,
+    layout,
+    out_channel_chunks,
+    out_channel_block,
+    out_channel_tail,
+    in_filter_channels,
+    in_data_channel_chunks,
+    in_data_channel_block,
+    in_data_channel_tail,
+    kernel_h,
+    kernel_w,
+):
+    """
+    Adds compute stages for packing of the filter in runtime. Extends channels dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Filter: tvm.te.Tensor
+        Filter tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    out_channel_chunks: int
+        Number of chunks for filters
+
+    out_channel_block: int
+        Size of the block
+
+    out_channel_tail: int
+        Original size of the latest chunk of output filters
+
+    in_filter_channels: int
+        Number of filter channels. might be different vs input channels in the
+        data due to groups/depthwise nature
+
+    in_data_channel_chunks: int
+        Number of chunks by channels for input data
+
+    in_data_channel_block: int
+        Size of the block for input data channels
+
+    in_data_channel_tail
+        Original size of the latest chunk for input data channels
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+    """
+    pad_value = tvm.tir.const(0, Filter.dtype)
+
+    def _reorder_weights_depthwise_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_depthwise_hwoi(*indices):
+        conditionA = []
+        conditionA.append(indices[2] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2] * out_channel_block + indices[4], indices[3]],
+        )
+
+    def _reorder_weights_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[1] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_hwio(*indices):
+        conditionA = []
+        conditionA.append(indices[3] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[2] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2], indices[3] * out_channel_block + indices[4]],
+        )
+
+    if in_filter_channels == 1:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_depthwise_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWOI":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, out_channel_chunks, in_filter_channels, out_channel_block],
+                _reorder_weights_depthwise_hwoi,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    else:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWIO":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block],
+                _reorder_weights_hwio,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    return reordered_filter
+
+
+def expand_spatial_dimensions(
+    in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+):
+    """
+    Expands spatial dimensions to be dividable by factor 4. This will allow us to do extrimely
+    better parallel computation on GPU. The drawback of this solution - it will be number of
+    useless computations. By fact the speed-up of parallelism significantly overcomes the slowdown
+    of extra compute and eventuially this is useful approach, at least for GPU
+
+    Parameters
+    ----------
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the featrue map

Review Comment:
   ```suggestion
           Width of the feature map
   ```



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )

Review Comment:
   Note: Explicit buffer layout padding as part of `transform_layout` is on the roadmap and will appear in RFC soon. Putting a note here to note that explicit layout transformations like this should be unnecessary in the future.



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865792339


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):

Review Comment:
   done



##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail

Review Comment:
   done



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

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

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


[GitHub] [tvm] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
tests/python/relay/test_conv2d_nchw_texture.py:
##########
@@ -0,0 +1,490 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):
+    mod_fp32 = recast(mod, "float32", "float32", ops=["nn.conv2d", "add", "nn.relu"])
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod_fp32, "llvm", params=params1)
+    ctx = tvm.cpu()
+    m = graph_runtime.create(graph, lib, ctx)
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            m.set_input(key, inputs[-1])
+    else:
+        m.set_input("data", inputs[-1])
+    m.set_input(**params)
+    m.run()
+    return [
+        m.get_output(0).asnumpy(),
+    ]
+
+
+# build module run with opencl and cpu, compare results
+def build_run_compare(
+    tvm_mod, params1, input_shape, dtype="float32", target="llvm", gpu_preprocess=None
+):
+
+    rpc_tracker_host = os.environ["TVM_TRACKER_HOST"]
+    rpc_tracker_port = os.environ["TVM_TRACKER_PORT"]
+    if rpc_tracker_host:
+        run_on_host = 0
+        target_host = "llvm -mtriple=arm64-linux-android"
+        rpc_tracker_port = int(rpc_tracker_port)
+    else:
+        run_on_host = 1
+        target_host = "llvm"
+
+    if gpu_preprocess:
+        tvm_mod_nchwc = gpu_preprocess(tvm_mod)
+    else:
+        tvm_mod_nchwc = tvm_mod
+
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(
+            tvm_mod_nchwc, target_host=target_host, target=target, params=params1
+        )
+    if run_on_host:
+        ctx = tvm.opencl()
+        m = graph_runtime.create(graph, lib, ctx)
+    else:
+        from tvm import rpc
+        from tvm.contrib import utils, ndk
+
+        rpc_key = "android"
+        tracker = rpc.connect_tracker(rpc_tracker_host, rpc_tracker_port)
+        remote = tracker.request(rpc_key, priority=0, session_timeout=600)
+        temp = utils.tempdir()
+        dso_binary = "dev_lib_cl.so"
+        dso_binary_path = temp.relpath(dso_binary)
+        ctx = remote.cl(0)
+        lib.export_library(dso_binary_path, ndk.create_shared)
+        remote.upload(dso_binary_path)
+        rlib = remote.load_module(dso_binary)
+        m = graph_runtime.create(graph, rlib, ctx)
+    m.set_input(**params)
+    inputs = []
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            inputs.append(np.random.normal(size=input_shape[key]).astype(dtype))
+            m.set_input(key, inputs[-1])
+    else:
+        inputs.append(np.random.normal(size=input_shape).astype(dtype))
+        m.set_input("data", inputs[-1])
+    m.run()
+
+    ref_outputs = get_reference(tvm_mod, params1, input_shape, inputs)
+    for i, ref_output in enumerate(ref_outputs):
+        tvm_output = m.get_output(i)
+        output = tvm_output.asnumpy()
+        # for index, x in np.ndenumerate(ref_output):
+        #     if abs(output[index] - x) > 0.01:
+        #         print(index, output[index], x)
+
+        np.testing.assert_allclose(output, ref_output, rtol=1e-1, atol=1e-1)
+
+
+def gpu_preprocess(tvm_mod):
+    layout_config = relay.transform.LayoutConfig()
+    desired_layouts = {"nn.conv2d": ["NCHW4c", "OIHW4o"]}
+    with layout_config:
+        seq = tvm.transform.Sequential([relay.transform.ConvertLayout(desired_layouts)])
+        with tvm.transform.PassContext(opt_level=3):
+            mod = tvm.IRModule.from_expr(tvm_mod)
+            tvm_mod_nchwc = seq(mod)
+            return tvm_mod_nchwc
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 42, 42)
+    filter_shape = (96, 32, 3, 3)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad_pass():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 40, 40)
+    filter_shape = (96, 32, 2, 2)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(2, 2),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_35_35_strides():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 48, 35, 35)
+    filter_shape = (64, 48, 5, 5)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[2, 2, 2, 2],
+        strides=[1, 1],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(5, 5),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_resnet50_v2_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 224, 224)
+    filter_shape = (64, 3, 7, 7)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 3, 3],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(7, 7),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    # mod, params = relay.testing.init.create_workload(func)
+    np.random.seed(1)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 299, 299)
+    filter_shape = (64, 3, 3, 3)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_1x1_16c16spatial():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 16, 256, 256)
+    filter_shape = (32, 16, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (32, 32, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (4, 32, 4, 4)
+    bias_shape = (1, 4, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=4,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_yolov3_v2_nchw_3c():

Review Comment:
   Do these tests pass on a local opencl device (e.g. with an nvgpu?). If not, it would be good to skip the tests that depend on the RPC tracker env vars if they are not set if they require a remote device. 



-- 
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] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r871318699


##########
python/tvm/topi/adreno/conv2d_nhwc.py:
##########
@@ -0,0 +1,339 @@
+# 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,no-else-return
+"""conv2d nhwc schedule on Qualcomm Adreno GPU"""
+import tvm
+from tvm import te
+from tvm import autotvm
+
+from ..utils import get_const_tuple, traverse_inline
+from .utils import (
+    split_to_chunks,
+    pack_input,
+    pack_filter,
+    expand_spatial_dimensions,
+    add_pad,
+    bind_data_copy,
+    get_texture_storage,
+)
+
+
+@autotvm.register_topi_compute("conv2d_nhwc.image2d")
+def conv2d_nhwc(cfg, data, kernel, strides, padding, dilation, out_dtype="float16"):
+    """Compute conv2d with NCHWc layout"""
+    args = {"shared": False, "accumulator": "float16"}
+    return compute_conv2d_NHWC_HWIO(data, kernel, strides, padding, dilation, out_dtype, args=args)
+
+
+@autotvm.register_topi_compute("conv2d_nhwc_acc32.image2d")
+def conv2d_nhwc_acc32(cfg, data, kernel, strides, padding, dilation, out_dtype="float16"):
+    """Compute conv2d with NCHWc layout"""
+    args = {"shared": False, "accumulator": "float32"}
+    return compute_conv2d_NHWC_HWIO(data, kernel, strides, padding, dilation, out_dtype, args=args)
+
+
+@autotvm.register_topi_schedule("conv2d_nhwc.image2d")
+def schedule_conv2d_nhwc(cfg, outs):
+    return schedule_conv2d_nhwc_impl(cfg, outs, tag="cast_from_acc16")
+
+
+@autotvm.register_topi_schedule("conv2d_nhwc_acc32.image2d")
+def schedule_conv2d_nhwc_acc32(cfg, outs):
+    return schedule_conv2d_nhwc_impl(cfg, outs, tag="cast_from_acc32")
+
+
+def schedule_conv2d_nhwc_impl(cfg, outs, tag):
+    """Create the schedule for conv2d_nhwc"""
+    outs = [outs] if isinstance(outs, te.tensor.Tensor) else outs
+    s = te.create_schedule([x.op for x in outs])
+
+    def _callback(op):
+        if op.tag == tag:
+            schedule_conv2d_NHWC(cfg, s, op.output(0))
+
+    traverse_inline(s, outs[0].op, _callback)
+    return s
+
+
+def compute_conv2d_NHWC_HWIO(Input, Filter, stride, padding, dilation, out_dtype, args):
+    """
+    Convolution operator in NHWC layout.
+    Algo:
+      1. Convert into blocked format if we have 4d original tensor.
+         In case of AutoTVM we override the convert by just tensors since such conversion
+         will be absent for real blocked convolution, no sense to include into tuning
+      2. Expand spatial dimensions to have width and height be dividable by factor 4
+         This leads to slightly bigger amount of compute but allow utilize GPU much better
+      3. Add paddings. This happens even if we do not need pad originaly. This is useful
+         due to work arounding of the gaps of texture annotation between Primary Functions
+         and limited support of textures in schedules. Later on this pad will be executed
+         separately and will produce texture
+      4. 5d Convolution compute with accumulating into out_dtype
+      5. Cast to the origin output data type
+      6. For case of 4d convolution: convert of output from 5d to 4d
+    """
+
+    if out_dtype is None:
+        out_dtype = Input.dtype
+    assert isinstance(stride, int) or len(stride) == 2
+    assert isinstance(dilation, int) or len(dilation) == 2
+    if isinstance(stride, int):
+        stride_h = stride_w = stride
+    else:
+        stride_h, stride_w = stride
+
+    if isinstance(dilation, int):
+        dilation_h = dilation_w = dilation
+    else:
+        dilation_h, dilation_w = dilation
+
+    convert_from4d = False
+    if len(Input.shape) == 4:
+        batch, in_height, in_width, in_channels = Input.shape
+        kernel_h, kernel_w, in_filter_channels, out_channles = Filter.shape
+
+        in_channel_chunks, in_channel_block, in_channel_tail = split_to_chunks(in_channels, 4)
+        out_channel_chunks, out_channel_block, out_channel_tail = split_to_chunks(out_channles, 4)
+
+        if autotvm.GLOBAL_SCOPE.in_tuning:
+            dshape = (batch, in_height, in_width, in_channel_chunks, in_channel_block)
+            Input = tvm.te.placeholder(dshape, Input.dtype, name="data_placeholder")
+            kshape = (kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block)
+            Filter = tvm.te.placeholder(kshape, Filter.dtype, name="kernel_placeholder")
+        else:
+            convert_from4d = True
+            Input = pack_input(
+                Input,
+                "NHWC",
+                batch,
+                in_channel_chunks,
+                in_channel_block,
+                in_channel_tail,
+                in_height,
+                in_width,
+            )
+            Filter = pack_filter(
+                Filter,
+                "HWIO",
+                out_channel_chunks,
+                out_channel_block,
+                out_channel_tail,
+                in_filter_channels,
+                in_channel_chunks,
+                in_channel_block,
+                in_channel_tail,
+                kernel_h,
+                kernel_w,
+            )
+
+    else:
+        batch, in_height, in_width, in_channel_chunks, in_channel_block = Input.shape
+        kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block = Filter.shape
+
+    out_height_orig, out_height, out_width_orig, out_width = expand_spatial_dimensions(
+        in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+    )
+
+    temp = add_pad(
+        Input,
+        "NHWC",
+        out_height_orig,
+        out_width_orig,
+        kernel_h,
+        kernel_w,
+        dilation_h,
+        dilation_w,
+        padding,
+        stride_h,
+        stride_w,
+    )
+
+    rcc = te.reduce_axis((0, in_channel_chunks), name="rcc")
+    rcb = te.reduce_axis((0, in_channel_block), name="rcb")
+    ry = te.reduce_axis((0, kernel_h), name="ry")
+    rx = te.reduce_axis((0, kernel_w), name="rx")
+    conv = te.compute(
+        (batch, out_height, out_width, out_channel_chunks, out_channel_block),
+        lambda nn, yy, xx, fc, fb: te.sum(
+            (
+                temp[nn, yy * stride_h + ry * dilation_h, xx * stride_w + rx * dilation_w, rcc, rcb]
+                * Filter[ry, rx, rcc * in_channel_block + rcb, fc, fb]
+            ).astype(args["accumulator"]),
+            axis=[ry, rx, rcc, rcb],
+        ),
+        tag="conv2d_nhwc",
+    )
+
+    if convert_from4d and not autotvm.GLOBAL_SCOPE.in_tuning:
+        dummy_cast = te.compute(
+            (batch, out_height_orig, out_width_orig, out_channel_chunks, out_channel_block),
+            lambda n, y, x, fc, fb: conv[n, y, x, fc, fb].astype(out_dtype),
+            tag="dummy_cast",
+        )
+        return te.compute(
+            (batch, out_height_orig, out_width_orig, out_channles),
+            lambda n, y, x, c: dummy_cast[n, y, x, c // out_channel_block, c % out_channel_block],
+            tag="cast_from_acc" + args["accumulator"][-2:],
+        )
+    else:
+        return te.compute(
+            (batch, out_height_orig, out_width_orig, out_channel_chunks, out_channel_block),
+            lambda n, y, x, ffc, ffb: conv[n, y, x, ffc, ffb].astype(out_dtype),
+            tag="cast_from_acc" + args["accumulator"][-2:],
+        )
+
+
+def schedule_conv2d_NHWC(cfg, s, output):
+    """
+    schedule optimized for batch size = 1
+
+    Algo:
+    1. Split output axis to three parts: global work size, vthread, local worksize.
+       The limitations for tuning includes heuristics from some tuned networks to limit
+       search space and not pay much time for useles configurations.
+    2. In case of 4d convolution schedule copying of the input (and filter) into
+      5d tensors
+    4. pad should be scheduled separately to create independent opencl kernel. If pad is
+       inlined into convolution, this gives 1.5x performance drop
+    5. We are using cache_read to produce texture and guarantee the best performance
+       on the next stage.
+    6. For 5d convolution we schedule the latest op with binding 5d axis and vectorize
+       for textures
+       For 4d tensor we are doing the same for the latest blocked stage, i.e. conversion
+       of data type
+    7. In case of 4d conv we need to schedule postops as well
+    """
+    latest = s.outputs[0].output(0)
+    if len(latest.op.axis) == 4:
+        latest_blocked = dummy = output.op.input_tensors[0]
+        conv = dummy.op.input_tensors[0]
+    else:
+        conv = output.op.input_tensors[0]
+        latest_blocked = latest
+
+    ##### space definition begin #####
+    n, y, x, fc, fb = s[conv].op.axis
+    ry, rx, rcc, rcb = s[conv].op.reduce_axis
+
+    if conv.shape[3] % 2 == 0:
+        min_threads_div = 2
+    else:
+        min_threads_div = 1
+
+    cfg.define_split(
+        "tile_fc",
+        fc,
+        num_outputs=3,
+        filter=lambda entity: entity.size[1] <= 8
+        and entity.size[2] >= min_threads_div
+        and entity.size[2] < 256,
+    )
+    cfg.define_split(
+        "tile_y",
+        y,
+        num_outputs=3,
+        filter=lambda entity: entity.size[1] <= 8 and entity.size[2] <= 16,
+    )
+    cfg.define_split(
+        "tile_x",
+        x,
+        num_outputs=3,
+        filter=lambda entity: entity.size[1] <= 8 and entity.size[2] <= 16,
+    )
+
+    cfg.define_split("tile_rcc", rcc, num_outputs=2)
+    cfg.define_split("tile_ry", ry, num_outputs=2)
+    cfg.define_split("tile_rx", rx, num_outputs=2)
+    cfg.define_knob("auto_unroll_max_step", [0, 512, 1500])
+    cfg.define_knob("unroll_explicit", [0, 1])
+
+    pad_data, kernel = s[conv].op.input_tensors
+    if (
+        isinstance(kernel.op, tvm.te.ComputeOp) and "filter_pack" in kernel.op.tag
+    ):  # len(latest.op.axis) == 4:
+        # manage scheduling of datacopy
+        pad_data, kernel = s[conv].op.input_tensors
+        pack_data = pad_data.op.input_tensors[0]
+        bind_data_copy(s[pack_data])
+        bind_data_copy(s[kernel])
+
+    pad_data, kernel = s[conv].op.input_tensors
+
+    s[pad_data].compute_inline()

Review Comment:
   It is inlined into next stage - cache read for textures
   ```
       AT = s.cache_read(pad_data, "global.texture", [conv])
       bind_data_copy(s[AT])
   ```
   If I do not add `s[pad_data].compute_inline()` the schedule would not be complete and would claim about missing of some bindings



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
src/runtime/opencl/opencl_common.h:
##########
@@ -345,6 +345,7 @@ struct BufferDescriptor {
      *         e.g. image2d[height=O, width=IHW]
      */
     kImage2DWeight,
+    kTexture2DNHWC,

Review Comment:
   I like that idea. Something like,
   
   > TODO(tvm-team): Uncouple use of storage scope and data layout by using the transform_layout schedule primitive to express the desired texture layout. This will require supporting Nd indices in BufferLoad and BufferStore in CodegenOpenCL, and ensuring Nd allocations for texture are correctly routed to the AllocateTexture packed function in the OpenCL DeviceAPI.



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
src/runtime/opencl/opencl_common.h:
##########
@@ -345,6 +345,7 @@ struct BufferDescriptor {
      *         e.g. image2d[height=O, width=IHW]
      */
     kImage2DWeight,
+    kTexture2DNHWC,

Review Comment:
   Note: We can now support arbitrary layouts with `transform_layout` which I will suggest we move to. It will require some rework on the TIR lowering. I don't suggest this block these schedules from being upstreamed now, but we should circle back on this soon. 



-- 
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] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865391417


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )
+
+    # compute:
+    if layout == "NCHW":
+        reordered_data = te.compute(
+            [batch, in_channel_chunks, in_height, in_width, in_channel_block],
+            _reorder_data_nchw,
+            name="input_pack",
+            tag="input_pack",
+        )
+    elif layout == "NHWC":
+        reordered_data = te.compute(
+            [batch, in_height, in_width, in_channel_chunks, in_channel_block],
+            _reorder_data_nhwc,
+            name="input_pack",
+            tag="input_pack",
+        )
+    else:
+        assert False, "Adreno util function pack_input does not accept unknown layout"
+    return reordered_data
+
+
+def pack_filter(
+    Filter,
+    layout,
+    out_channel_chunks,
+    out_channel_block,
+    out_channel_tail,
+    in_filter_channels,
+    in_data_channel_chunks,
+    in_data_channel_block,
+    in_data_channel_tail,
+    kernel_h,
+    kernel_w,
+):
+    """
+    Adds compute stages for packing of the filter in runtime. Extends channels dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Filter: tvm.te.Tensor
+        Filter tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    out_channel_chunks: int
+        Number of chunks for filters
+
+    out_channel_block: int
+        Size of the block
+
+    out_channel_tail: int
+        Original size of the latest chunk of output filters
+
+    in_filter_channels: int
+        Number of filter channels. might be different vs input channels in the
+        data due to groups/depthwise nature
+
+    in_data_channel_chunks: int
+        Number of chunks by channels for input data
+
+    in_data_channel_block: int
+        Size of the block for input data channels
+
+    in_data_channel_tail
+        Original size of the latest chunk for input data channels
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+    """
+    pad_value = tvm.tir.const(0, Filter.dtype)
+
+    def _reorder_weights_depthwise_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_depthwise_hwoi(*indices):
+        conditionA = []
+        conditionA.append(indices[2] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2] * out_channel_block + indices[4], indices[3]],
+        )
+
+    def _reorder_weights_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[1] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_hwio(*indices):
+        conditionA = []
+        conditionA.append(indices[3] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[2] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2], indices[3] * out_channel_block + indices[4]],
+        )
+
+    if in_filter_channels == 1:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_depthwise_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWOI":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, out_channel_chunks, in_filter_channels, out_channel_block],
+                _reorder_weights_depthwise_hwoi,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    else:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWIO":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block],
+                _reorder_weights_hwio,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    return reordered_filter
+
+
+def expand_spatial_dimensions(
+    in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+):
+    """
+    Expands spatial dimensions to be dividable by factor 4. This will allow us to do extrimely

Review Comment:
   could you please point where typos are?



-- 
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] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865794798


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )
+
+    # compute:
+    if layout == "NCHW":
+        reordered_data = te.compute(
+            [batch, in_channel_chunks, in_height, in_width, in_channel_block],
+            _reorder_data_nchw,
+            name="input_pack",
+            tag="input_pack",
+        )
+    elif layout == "NHWC":
+        reordered_data = te.compute(
+            [batch, in_height, in_width, in_channel_chunks, in_channel_block],
+            _reorder_data_nhwc,
+            name="input_pack",
+            tag="input_pack",
+        )
+    else:
+        assert False, "Adreno util function pack_input does not accept unknown layout"
+    return reordered_data
+
+
+def pack_filter(
+    Filter,
+    layout,
+    out_channel_chunks,
+    out_channel_block,
+    out_channel_tail,
+    in_filter_channels,
+    in_data_channel_chunks,
+    in_data_channel_block,
+    in_data_channel_tail,
+    kernel_h,
+    kernel_w,
+):
+    """
+    Adds compute stages for packing of the filter in runtime. Extends channels dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Filter: tvm.te.Tensor
+        Filter tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    out_channel_chunks: int
+        Number of chunks for filters
+
+    out_channel_block: int
+        Size of the block
+
+    out_channel_tail: int
+        Original size of the latest chunk of output filters
+
+    in_filter_channels: int
+        Number of filter channels. might be different vs input channels in the
+        data due to groups/depthwise nature
+
+    in_data_channel_chunks: int
+        Number of chunks by channels for input data
+
+    in_data_channel_block: int
+        Size of the block for input data channels
+
+    in_data_channel_tail
+        Original size of the latest chunk for input data channels
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+    """
+    pad_value = tvm.tir.const(0, Filter.dtype)
+
+    def _reorder_weights_depthwise_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_depthwise_hwoi(*indices):
+        conditionA = []
+        conditionA.append(indices[2] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2] * out_channel_block + indices[4], indices[3]],
+        )
+
+    def _reorder_weights_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[1] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_hwio(*indices):
+        conditionA = []
+        conditionA.append(indices[3] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[2] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2], indices[3] * out_channel_block + indices[4]],
+        )
+
+    if in_filter_channels == 1:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_depthwise_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWOI":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, out_channel_chunks, in_filter_channels, out_channel_block],
+                _reorder_weights_depthwise_hwoi,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    else:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWIO":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block],
+                _reorder_weights_hwio,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    return reordered_filter
+
+
+def expand_spatial_dimensions(
+    in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+):
+    """
+    Expands spatial dimensions to be dividable by factor 4. This will allow us to do extrimely
+    better parallel computation on GPU. The drawback of this solution - it will be number of
+    useless computations. By fact the speed-up of parallelism significantly overcomes the slowdown
+    of extra compute and eventuially this is useful approach, at least for GPU
+
+    Parameters
+    ----------
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the featrue map
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+
+    dilation_h: int
+        Vertical dilation of the conv2d kernel
+
+    dilation_w: int
+        Horizontal dilation of the conv2d kernel
+
+    padding: tuple or list
+        Conv2d paddings
+
+    stride_h: int
+        Vertical stride  of the conv2d kernel
+
+    stride_w: int
+        Horizontal stride  of the conv2d kernel
+    """
+    dilated_kernel_h = (kernel_h - 1) * dilation_h + 1
+    dilated_kernel_w = (kernel_w - 1) * dilation_w + 1
+
+    pad_top, pad_left, pad_down, pad_right = nn.get_pad_tuple(
+        padding, (dilated_kernel_h, dilated_kernel_w)
+    )
+
+    out_height_orig = out_height = simplify(
+        (in_height - dilated_kernel_h + pad_top + pad_down) // stride_h + 1
+    )
+    out_width_orig = out_width = simplify(
+        (in_width - dilated_kernel_w + pad_left + pad_right) // stride_w + 1
+    )
+
+    # can output shape be divded by 2 or even 4?
+    # if it cannot be divided, need to extend for further help with split
+    # theortically there should be addition padding for inputs, but it will be optimized by
+    # cache_read InferBound. We must proceed pad here exactly to produce tensor which is
+    # required for calculation of original out size, not more! In other case intermediate
+    # tensor might be allcoated with less sizes while compute will try to fill the expanded
+    # one - data discrepancy as a result
+    # And in case of textures it is not a problem if we provide texture of less size because
+    # 1. It is not important which values would be for extra calc - these calculations are
+    #    required only for better utilizatin of GPU fit to working groups
+    # 2. When we request pixel out opf bound, texture will handle this correctly. As mentioned
+    #    above, the value itself is not important
+    if out_height % 2 != 0:
+        out_height += 1
+    if out_width % 2 != 0:
+        out_width += 1
+
+    if out_height % 4 != 0:
+        out_height += 2
+    if out_width % 4 != 0:
+        out_width += 2
+    return out_height_orig, out_height, out_width_orig, out_width
+
+
+def add_pad(
+    data,
+    layout,
+    out_height,
+    out_width,
+    kernel_h,
+    kernel_w,
+    dilation_h,
+    dilation_w,
+    padding,
+    stride_h,
+    stride_w,
+):
+    """Computes required padding values by the parameters of conv2d and adds
+        compute for extending of original tensor
+
+    Parameters
+    ----------
+    data: tvm.te.Tensor
+        5d tensor, the layout of spatial dimensions are defined as separate argument
+
+    layout: string
+        Layout of origin 4d tensor
+
+    out_height: int
+        Height of the output feature map
+
+    out_width: int
+        Width of the output feature map
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+
+    dilation_h: int
+        Height dilation value from conv2d attributes
+
+    dilation_w: int
+        Width dilation value from conv2d attributes
+
+    padding: list / tuple of n ints
+        Padding values from conv2d attributes
+
+    stride_h: int
+        Height stride value from conv2d attributes
+
+    stride_w: int
+        Width stride value from conv2d attributes
+
+    Returns
+    -------
+    Output : tvm.te.Tensor
+        n-D, the same layout as Input.
+    """
+    dilated_kernel_h = (kernel_h - 1) * dilation_h + 1
+    dilated_kernel_w = (kernel_w - 1) * dilation_w + 1
+    pad_top, pad_left, pad_down, pad_right = nn.get_pad_tuple(
+        padding, (dilated_kernel_h, dilated_kernel_w)
+    )
+
+    # compute graph
+    if layout == "NCHW":
+        y_axis = 2
+        x_axis = 3
+        if len(data.shape) == 4:
+            _, _, in_height, in_width = data.shape
+        else:
+            _, _, in_height, in_width, _ = data.shape
+    elif layout == "NHWC":
+        y_axis = 1
+        x_axis = 2
+        if len(data.shape) == 4:
+            _, in_height, in_width, _ = data.shape
+        else:
+            _, in_height, in_width, _, _ = data.shape
+    else:
+        assert False, "not supported layout in adreno util add_pad"
+    pad_before = [0, 0, 0, 0, 0]
+    pad_after = [0, 0, 0, 0, 0]
+    pad_before[y_axis] = pad_top
+    pad_before[x_axis] = pad_left
+    pad_after[y_axis] = pad_down
+    pad_after[x_axis] = pad_right
+
+    # calculation of real used input size:
+    input_latest_w = (out_width - 1) * stride_w + (kernel_w - 1) * dilation_w + 1
+    input_latest_h = (out_height - 1) * stride_h + (kernel_h - 1) * dilation_h + 1
+    if input_latest_w < in_width + pad_before[x_axis] + pad_after[x_axis]:
+        pad_after[x_axis] -= in_width + pad_before[x_axis] + pad_after[x_axis] - input_latest_w
+    if input_latest_h < in_height + pad_before[y_axis] + pad_after[y_axis]:
+        pad_after[y_axis] -= in_height + pad_before[y_axis] + pad_after[y_axis] - input_latest_h
+    return nn.pad(data, pad_before, pad_after, name="pad_temp")
+
+
+def bind_data_copy(stage, axis_to_vectorize=None):
+    """
+    Schedules the eltwise stages like copying of data or postops
+
+    Parameters
+    ----------
+    stage: tvm.te.Tensor
+
+    axis_to_vectorize:
+        Causes to split certain axis, moves inner part to the end of schedule
+        and enable vectorization by this axis
+        If parameter is not pointed, the schedule will be vectorized if the most inner
+        dim is eq to 4 (size of the vector in texture)
+    """
+    shape = get_const_tuple(stage.op.output(0).shape)
+    if axis_to_vectorize and len(shape) == 4 and shape[axis_to_vectorize] % 4 == 0:
+        ax0, ax1, ax2, ax3 = stage.op.axis
+        if axis_to_vectorize == 1:
+            oax1, iax1 = stage.split(ax1, factor=4)
+            stage.reorder(ax0, oax1, ax2, ax3, iax1)
+            stage.vectorize(iax1)
+            fused = stage.fuse(ax0, oax1, ax2, ax3)
+        elif axis_to_vectorize == 3:
+            oax3, iax3 = stage.split(ax3, factor=4)
+            stage.reorder(ax0, ax1, ax2, oax3, iax3)
+            stage.vectorize(iax3)
+            fused = stage.fuse(ax0, ax1, ax2, oax3)
+
+        ftc = numpy.prod(shape) / 4
+        div = getDiv(ftc, 128)
+        block, thread = stage.split(fused, factor=div)
+
+        stage.bind(block, te.thread_axis("blockIdx.z"))
+        stage.bind(thread, te.thread_axis("threadIdx.z"))
+    else:
+        axes = stage.op.axis
+        fused = stage.fuse(*axes[:-1])
+        if shape[-1] <= 32:
+            ftc = numpy.prod(shape[:-1])
+            div = getDiv(ftc, 64)
+            block, thread = stage.split(fused, factor=div)
+            stage.bind(block, te.thread_axis("blockIdx.x"))
+            stage.bind(thread, te.thread_axis("threadIdx.x"))
+            if shape[-1] == 4:
+                stage.vectorize(axes[-1])
+        else:
+            stage.bind(fused, te.thread_axis("blockIdx.x"))
+            stage.bind(*axes[-1:], te.thread_axis("threadIdx.x"))
+
+
+def get_texture_storage(shape):
+    """
+    Returns the texture layout acceptable for the shape
+
+    Parameters
+    ----------
+    shape: array
+        Shape of the tensor to be packed to texture
+    """
+    # certain limitation of the Qualcomm devices. Subject to be determined for certain device
+    # individually, but until we have access to remote device during compilation, we have to
+    # define it uniformly for all target devices
+    limit = 16384

Review Comment:
   I added new `texture_spatial_limit` attribute to opencl target, added to the `DeviceAttrKind` and runtime_ctypes in python, but not sure if it was required since I don;t know how and when to use [DetectDeviceFlag](https://github.com/apache/tvm/blob/main/src/target/target_kind.cc#L114) as well I have an access to the `texture_spatial_limit` in the python part through `tvm.target.Target.current().attrs["texture_spatial_limit"]`
   I would consider this as "addressed" but need to understand if my solution is applicable and if we need parts related to `DeviceAttrKind`



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
python/tvm/topi/adreno/conv2d_nhwc.py:
##########
@@ -0,0 +1,339 @@
+# 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,no-else-return
+"""conv2d nhwc schedule on Qualcomm Adreno GPU"""
+import tvm
+from tvm import te
+from tvm import autotvm
+
+from ..utils import get_const_tuple, traverse_inline
+from .utils import (
+    split_to_chunks,
+    pack_input,
+    pack_filter,
+    expand_spatial_dimensions,
+    add_pad,
+    bind_data_copy,
+    get_texture_storage,
+)
+
+
+@autotvm.register_topi_compute("conv2d_nhwc.image2d")
+def conv2d_nhwc(cfg, data, kernel, strides, padding, dilation, out_dtype="float16"):
+    """Compute conv2d with NCHWc layout"""
+    args = {"shared": False, "accumulator": "float16"}
+    return compute_conv2d_NHWC_HWIO(data, kernel, strides, padding, dilation, out_dtype, args=args)
+
+
+@autotvm.register_topi_compute("conv2d_nhwc_acc32.image2d")
+def conv2d_nhwc_acc32(cfg, data, kernel, strides, padding, dilation, out_dtype="float16"):
+    """Compute conv2d with NCHWc layout"""
+    args = {"shared": False, "accumulator": "float32"}
+    return compute_conv2d_NHWC_HWIO(data, kernel, strides, padding, dilation, out_dtype, args=args)
+
+
+@autotvm.register_topi_schedule("conv2d_nhwc.image2d")
+def schedule_conv2d_nhwc(cfg, outs):
+    return schedule_conv2d_nhwc_impl(cfg, outs, tag="cast_from_acc16")
+
+
+@autotvm.register_topi_schedule("conv2d_nhwc_acc32.image2d")
+def schedule_conv2d_nhwc_acc32(cfg, outs):
+    return schedule_conv2d_nhwc_impl(cfg, outs, tag="cast_from_acc32")
+
+
+def schedule_conv2d_nhwc_impl(cfg, outs, tag):
+    """Create the schedule for conv2d_nhwc"""
+    outs = [outs] if isinstance(outs, te.tensor.Tensor) else outs
+    s = te.create_schedule([x.op for x in outs])
+
+    def _callback(op):
+        if op.tag == tag:
+            schedule_conv2d_NHWC(cfg, s, op.output(0))
+
+    traverse_inline(s, outs[0].op, _callback)
+    return s
+
+
+def compute_conv2d_NHWC_HWIO(Input, Filter, stride, padding, dilation, out_dtype, args):
+    """
+    Convolution operator in NHWC layout.
+    Algo:
+      1. Convert into blocked format if we have 4d original tensor.
+         In case of AutoTVM we override the convert by just tensors since such conversion
+         will be absent for real blocked convolution, no sense to include into tuning
+      2. Expand spatial dimensions to have width and height be dividable by factor 4
+         This leads to slightly bigger amount of compute but allow utilize GPU much better
+      3. Add paddings. This happens even if we do not need pad originaly. This is useful
+         due to work arounding of the gaps of texture annotation between Primary Functions
+         and limited support of textures in schedules. Later on this pad will be executed
+         separately and will produce texture
+      4. 5d Convolution compute with accumulating into out_dtype
+      5. Cast to the origin output data type
+      6. For case of 4d convolution: convert of output from 5d to 4d
+    """
+
+    if out_dtype is None:
+        out_dtype = Input.dtype
+    assert isinstance(stride, int) or len(stride) == 2
+    assert isinstance(dilation, int) or len(dilation) == 2
+    if isinstance(stride, int):
+        stride_h = stride_w = stride
+    else:
+        stride_h, stride_w = stride
+
+    if isinstance(dilation, int):
+        dilation_h = dilation_w = dilation
+    else:
+        dilation_h, dilation_w = dilation
+
+    convert_from4d = False
+    if len(Input.shape) == 4:
+        batch, in_height, in_width, in_channels = Input.shape
+        kernel_h, kernel_w, in_filter_channels, out_channles = Filter.shape
+
+        in_channel_chunks, in_channel_block, in_channel_tail = split_to_chunks(in_channels, 4)
+        out_channel_chunks, out_channel_block, out_channel_tail = split_to_chunks(out_channles, 4)
+
+        if autotvm.GLOBAL_SCOPE.in_tuning:
+            dshape = (batch, in_height, in_width, in_channel_chunks, in_channel_block)
+            Input = tvm.te.placeholder(dshape, Input.dtype, name="data_placeholder")
+            kshape = (kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block)
+            Filter = tvm.te.placeholder(kshape, Filter.dtype, name="kernel_placeholder")
+        else:
+            convert_from4d = True
+            Input = pack_input(
+                Input,
+                "NHWC",
+                batch,
+                in_channel_chunks,
+                in_channel_block,
+                in_channel_tail,
+                in_height,
+                in_width,
+            )
+            Filter = pack_filter(
+                Filter,
+                "HWIO",
+                out_channel_chunks,
+                out_channel_block,
+                out_channel_tail,
+                in_filter_channels,
+                in_channel_chunks,
+                in_channel_block,
+                in_channel_tail,
+                kernel_h,
+                kernel_w,
+            )
+
+    else:
+        batch, in_height, in_width, in_channel_chunks, in_channel_block = Input.shape
+        kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block = Filter.shape
+
+    out_height_orig, out_height, out_width_orig, out_width = expand_spatial_dimensions(
+        in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+    )
+
+    temp = add_pad(
+        Input,
+        "NHWC",
+        out_height_orig,
+        out_width_orig,
+        kernel_h,
+        kernel_w,
+        dilation_h,
+        dilation_w,
+        padding,
+        stride_h,
+        stride_w,
+    )
+
+    rcc = te.reduce_axis((0, in_channel_chunks), name="rcc")
+    rcb = te.reduce_axis((0, in_channel_block), name="rcb")
+    ry = te.reduce_axis((0, kernel_h), name="ry")
+    rx = te.reduce_axis((0, kernel_w), name="rx")
+    conv = te.compute(
+        (batch, out_height, out_width, out_channel_chunks, out_channel_block),
+        lambda nn, yy, xx, fc, fb: te.sum(
+            (
+                temp[nn, yy * stride_h + ry * dilation_h, xx * stride_w + rx * dilation_w, rcc, rcb]
+                * Filter[ry, rx, rcc * in_channel_block + rcb, fc, fb]
+            ).astype(args["accumulator"]),
+            axis=[ry, rx, rcc, rcb],
+        ),
+        tag="conv2d_nhwc",
+    )
+
+    if convert_from4d and not autotvm.GLOBAL_SCOPE.in_tuning:
+        dummy_cast = te.compute(
+            (batch, out_height_orig, out_width_orig, out_channel_chunks, out_channel_block),
+            lambda n, y, x, fc, fb: conv[n, y, x, fc, fb].astype(out_dtype),
+            tag="dummy_cast",
+        )
+        return te.compute(
+            (batch, out_height_orig, out_width_orig, out_channles),
+            lambda n, y, x, c: dummy_cast[n, y, x, c // out_channel_block, c % out_channel_block],
+            tag="cast_from_acc" + args["accumulator"][-2:],
+        )
+    else:
+        return te.compute(
+            (batch, out_height_orig, out_width_orig, out_channel_chunks, out_channel_block),
+            lambda n, y, x, ffc, ffb: conv[n, y, x, ffc, ffb].astype(out_dtype),
+            tag="cast_from_acc" + args["accumulator"][-2:],
+        )
+
+
+def schedule_conv2d_NHWC(cfg, s, output):
+    """
+    schedule optimized for batch size = 1
+
+    Algo:
+    1. Split output axis to three parts: global work size, vthread, local worksize.
+       The limitations for tuning includes heuristics from some tuned networks to limit
+       search space and not pay much time for useles configurations.
+    2. In case of 4d convolution schedule copying of the input (and filter) into
+      5d tensors
+    4. pad should be scheduled separately to create independent opencl kernel. If pad is
+       inlined into convolution, this gives 1.5x performance drop
+    5. We are using cache_read to produce texture and guarantee the best performance
+       on the next stage.
+    6. For 5d convolution we schedule the latest op with binding 5d axis and vectorize
+       for textures
+       For 4d tensor we are doing the same for the latest blocked stage, i.e. conversion
+       of data type
+    7. In case of 4d conv we need to schedule postops as well
+    """
+    latest = s.outputs[0].output(0)
+    if len(latest.op.axis) == 4:
+        latest_blocked = dummy = output.op.input_tensors[0]
+        conv = dummy.op.input_tensors[0]
+    else:
+        conv = output.op.input_tensors[0]
+        latest_blocked = latest
+
+    ##### space definition begin #####
+    n, y, x, fc, fb = s[conv].op.axis
+    ry, rx, rcc, rcb = s[conv].op.reduce_axis
+
+    if conv.shape[3] % 2 == 0:
+        min_threads_div = 2
+    else:
+        min_threads_div = 1
+
+    cfg.define_split(
+        "tile_fc",
+        fc,
+        num_outputs=3,
+        filter=lambda entity: entity.size[1] <= 8
+        and entity.size[2] >= min_threads_div
+        and entity.size[2] < 256,
+    )
+    cfg.define_split(
+        "tile_y",
+        y,
+        num_outputs=3,
+        filter=lambda entity: entity.size[1] <= 8 and entity.size[2] <= 16,
+    )
+    cfg.define_split(
+        "tile_x",
+        x,
+        num_outputs=3,
+        filter=lambda entity: entity.size[1] <= 8 and entity.size[2] <= 16,
+    )
+
+    cfg.define_split("tile_rcc", rcc, num_outputs=2)
+    cfg.define_split("tile_ry", ry, num_outputs=2)
+    cfg.define_split("tile_rx", rx, num_outputs=2)
+    cfg.define_knob("auto_unroll_max_step", [0, 512, 1500])
+    cfg.define_knob("unroll_explicit", [0, 1])
+
+    pad_data, kernel = s[conv].op.input_tensors
+    if (
+        isinstance(kernel.op, tvm.te.ComputeOp) and "filter_pack" in kernel.op.tag
+    ):  # len(latest.op.axis) == 4:
+        # manage scheduling of datacopy
+        pad_data, kernel = s[conv].op.input_tensors
+        pack_data = pad_data.op.input_tensors[0]
+        bind_data_copy(s[pack_data])
+        bind_data_copy(s[kernel])
+
+    pad_data, kernel = s[conv].op.input_tensors
+
+    s[pad_data].compute_inline()

Review Comment:
   Are you meaning to inline padding here? Your comment [above](https://github.com/apache/tvm/pull/11161/files#diff-b3830c92b31f9a929f0eccbb82bde0013f49737d9f98424641d91a5435a86bfcR210) implies that you intend to do otherwise.



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
tests/python/relay/test_conv2d_nchw_texture.py:
##########
@@ -0,0 +1,490 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):
+    mod_fp32 = recast(mod, "float32", "float32", ops=["nn.conv2d", "add", "nn.relu"])
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod_fp32, "llvm", params=params1)
+    ctx = tvm.cpu()
+    m = graph_runtime.create(graph, lib, ctx)
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            m.set_input(key, inputs[-1])
+    else:
+        m.set_input("data", inputs[-1])
+    m.set_input(**params)
+    m.run()
+    return [
+        m.get_output(0).asnumpy(),
+    ]
+
+
+# build module run with opencl and cpu, compare results
+def build_run_compare(

Review Comment:
   Common utility



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
src/target/target_kind.cc:
##########
@@ -324,6 +324,7 @@ TVM_REGISTER_TARGET_KIND("opencl", kDLOpenCL)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
     .add_attr_option<Integer>("thread_warp_size", Integer(1))
+    .add_attr_option<Integer>("texture_spatial_limit", Integer(16384))

Review Comment:
   Thanks for adding this. An improvement would be to query the remote device using a call to the device api GetAttr using the target attr preprocessor.



-- 
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] csullivan commented on pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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

   Many thanks for the great work @elvin-n, @echuraev, @lhez. This is merged.


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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865391568


##########
python/tvm/topi/adreno/utils.py:
##########
@@ -0,0 +1,545 @@
+# 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,no-else-return
+"""util functions to be reused in different compute/schedule on Qualcomm Adreno GPU"""
+
+import tvm
+import numpy
+from tvm import te
+from tvm.topi.utils import simplify
+from tvm.topi import nn
+from ..utils import get_const_tuple
+
+
+def getDiv(value, start):
+    """Returns the maximum divider for `value` starting from `start` value"""
+    div = 1
+    for d in range(start, 0, -1):
+        if (value % d) == 0:
+            div = d
+            break
+    return div
+
+
+def split_to_chunks(trip_count, block):
+    """
+    Splits the trip count value to chunks and block, returns the remainder as well
+    the chunks and blocks covers or overlaps the origin trip_count
+
+    If trip_count can be divisible by block:
+        trip_count = chunks * block
+    else
+        trip_count = (chunks - 1) * block + tail
+
+    Parameters
+    ----------
+    trip_count: int
+        tripcount for original compute
+
+    block: int
+        size of the block
+
+    Returns
+    ----------
+    out: tuple of the (chunks, block, tail)
+    """
+    tail = trip_count % 4
+    chunks = trip_count // 4
+    if tail == 0:
+        tail = 4
+    else:
+        chunks += 1
+    return chunks, block, tail
+
+
+def pack_input(
+    Input, layout, batch, in_channel_chunks, in_channel_block, in_channel_tail, in_height, in_width
+):
+    """
+    Adds compute stages for packing of the data in runtime. Extends channel dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Input: tvm.te.Tensor
+        Input tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    batch: int
+        Batch size
+
+    in_channel_chunks: int
+        Number of channel chunks been in the final tensor
+
+    in_channel_block: int
+        Number of channel blocks been in the final tensor
+
+    in_channel_tail: int
+        Tail in the latest chunk diffing original number of channels vs blocked one
+        If in_channel_tail != in_channel_block:
+          original_channels = in_channel_chunks * in_channel_block - in_channel_tail
+        else
+          original_channels = in_channel_chunks * in_channel_block
+
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the feature map
+    """
+
+    pad_value = tvm.tir.const(0, Input.dtype)
+
+    def _reorder_data_nchw(*indices):
+        condition = []
+        condition.append(indices[1] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1] * in_channel_block + indices[4], indices[2], indices[3]],
+        )
+
+    def _reorder_data_nhwc(*indices):
+        condition = []
+        condition.append(indices[3] == in_channel_chunks - 1)
+        condition.append(indices[4] >= in_channel_tail)
+        condition = tvm.tir.all(*condition)
+        return tvm.tir.if_then_else(
+            condition,
+            pad_value,
+            Input[indices[0], indices[1], indices[2], indices[3] * in_channel_block + indices[4]],
+        )
+
+    # compute:
+    if layout == "NCHW":
+        reordered_data = te.compute(
+            [batch, in_channel_chunks, in_height, in_width, in_channel_block],
+            _reorder_data_nchw,
+            name="input_pack",
+            tag="input_pack",
+        )
+    elif layout == "NHWC":
+        reordered_data = te.compute(
+            [batch, in_height, in_width, in_channel_chunks, in_channel_block],
+            _reorder_data_nhwc,
+            name="input_pack",
+            tag="input_pack",
+        )
+    else:
+        assert False, "Adreno util function pack_input does not accept unknown layout"
+    return reordered_data
+
+
+def pack_filter(
+    Filter,
+    layout,
+    out_channel_chunks,
+    out_channel_block,
+    out_channel_tail,
+    in_filter_channels,
+    in_data_channel_chunks,
+    in_data_channel_block,
+    in_data_channel_tail,
+    kernel_h,
+    kernel_w,
+):
+    """
+    Adds compute stages for packing of the filter in runtime. Extends channels dimensions
+    to be dividable by factor 4
+
+    Parameters
+    ----------
+    Filter: tvm.te.Tensor
+        Filter tensor to be repacked in runtime
+
+    layout: string
+        Layout of origin 4d tensor
+        NCHW or NHWC are acceptable
+
+    out_channel_chunks: int
+        Number of chunks for filters
+
+    out_channel_block: int
+        Size of the block
+
+    out_channel_tail: int
+        Original size of the latest chunk of output filters
+
+    in_filter_channels: int
+        Number of filter channels. might be different vs input channels in the
+        data due to groups/depthwise nature
+
+    in_data_channel_chunks: int
+        Number of chunks by channels for input data
+
+    in_data_channel_block: int
+        Size of the block for input data channels
+
+    in_data_channel_tail
+        Original size of the latest chunk for input data channels
+
+    kernel_h: int
+        Height of the conv2d kernel
+
+    kernel_w: int
+        Width of the conv2d kernel
+    """
+    pad_value = tvm.tir.const(0, Filter.dtype)
+
+    def _reorder_weights_depthwise_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_depthwise_hwoi(*indices):
+        conditionA = []
+        conditionA.append(indices[2] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        return tvm.tir.if_then_else(
+            conditionAT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2] * out_channel_block + indices[4], indices[3]],
+        )
+
+    def _reorder_weights_oihw(*indices):
+        conditionA = []
+        conditionA.append(indices[0] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[1] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0] * out_channel_block + indices[4], indices[1], indices[2], indices[3]],
+        )
+
+    def _reorder_weights_hwio(*indices):
+        conditionA = []
+        conditionA.append(indices[3] == out_channel_chunks - 1)
+        conditionA.append(indices[4] >= out_channel_tail)
+        conditionAT = tvm.tir.all(*conditionA)
+
+        conditionO = []
+        conditionO.append(conditionAT)
+        conditionO.append(
+            indices[2] >= in_data_channel_chunks * in_data_channel_block + in_data_channel_tail
+        )
+        conditionOT = tvm.tir.any(*conditionO)
+        return tvm.tir.if_then_else(
+            conditionOT,
+            pad_value,
+            Filter[indices[0], indices[1], indices[2], indices[3] * out_channel_block + indices[4]],
+        )
+
+    if in_filter_channels == 1:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_depthwise_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWOI":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, out_channel_chunks, in_filter_channels, out_channel_block],
+                _reorder_weights_depthwise_hwoi,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    else:
+        if layout == "OIHW":
+            reordered_filter = te.compute(
+                [out_channel_chunks, in_filter_channels, kernel_h, kernel_w, out_channel_block],
+                _reorder_weights_oihw,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        elif layout == "HWIO":
+            reordered_filter = te.compute(
+                [kernel_h, kernel_w, in_filter_channels, out_channel_chunks, out_channel_block],
+                _reorder_weights_hwio,
+                name="filter_pack",
+                tag="filter_pack",
+            )
+        else:
+            assert False, "Adreno util function def pack_filter does not accept unknown layout"
+    return reordered_filter
+
+
+def expand_spatial_dimensions(
+    in_height, in_width, kernel_h, kernel_w, dilation_h, dilation_w, padding, stride_h, stride_w
+):
+    """
+    Expands spatial dimensions to be dividable by factor 4. This will allow us to do extrimely
+    better parallel computation on GPU. The drawback of this solution - it will be number of
+    useless computations. By fact the speed-up of parallelism significantly overcomes the slowdown
+    of extra compute and eventuially this is useful approach, at least for GPU
+
+    Parameters
+    ----------
+    in_height: int
+        Height of the feature map
+
+    in_width: int
+        Width of the featrue map

Review Comment:
   done



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r872229637


##########
tests/python/relay/test_conv2d_nchw_texture.py:
##########
@@ -0,0 +1,490 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):
+    mod_fp32 = recast(mod, "float32", "float32", ops=["nn.conv2d", "add", "nn.relu"])
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod_fp32, "llvm", params=params1)
+    ctx = tvm.cpu()
+    m = graph_runtime.create(graph, lib, ctx)
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            m.set_input(key, inputs[-1])
+    else:
+        m.set_input("data", inputs[-1])
+    m.set_input(**params)
+    m.run()
+    return [
+        m.get_output(0).asnumpy(),
+    ]
+
+
+# build module run with opencl and cpu, compare results
+def build_run_compare(
+    tvm_mod, params1, input_shape, dtype="float32", target="llvm", gpu_preprocess=None
+):
+
+    rpc_tracker_host = os.environ["TVM_TRACKER_HOST"]
+    rpc_tracker_port = os.environ["TVM_TRACKER_PORT"]
+    if rpc_tracker_host:
+        run_on_host = 0
+        target_host = "llvm -mtriple=arm64-linux-android"
+        rpc_tracker_port = int(rpc_tracker_port)
+    else:
+        run_on_host = 1
+        target_host = "llvm"
+
+    if gpu_preprocess:
+        tvm_mod_nchwc = gpu_preprocess(tvm_mod)
+    else:
+        tvm_mod_nchwc = tvm_mod
+
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(
+            tvm_mod_nchwc, target_host=target_host, target=target, params=params1
+        )
+    if run_on_host:
+        ctx = tvm.opencl()
+        m = graph_runtime.create(graph, lib, ctx)
+    else:
+        from tvm import rpc
+        from tvm.contrib import utils, ndk
+
+        rpc_key = "android"
+        tracker = rpc.connect_tracker(rpc_tracker_host, rpc_tracker_port)
+        remote = tracker.request(rpc_key, priority=0, session_timeout=600)
+        temp = utils.tempdir()
+        dso_binary = "dev_lib_cl.so"
+        dso_binary_path = temp.relpath(dso_binary)
+        ctx = remote.cl(0)
+        lib.export_library(dso_binary_path, ndk.create_shared)
+        remote.upload(dso_binary_path)
+        rlib = remote.load_module(dso_binary)
+        m = graph_runtime.create(graph, rlib, ctx)
+    m.set_input(**params)
+    inputs = []
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            inputs.append(np.random.normal(size=input_shape[key]).astype(dtype))
+            m.set_input(key, inputs[-1])
+    else:
+        inputs.append(np.random.normal(size=input_shape).astype(dtype))
+        m.set_input("data", inputs[-1])
+    m.run()
+
+    ref_outputs = get_reference(tvm_mod, params1, input_shape, inputs)
+    for i, ref_output in enumerate(ref_outputs):
+        tvm_output = m.get_output(i)
+        output = tvm_output.asnumpy()
+        # for index, x in np.ndenumerate(ref_output):
+        #     if abs(output[index] - x) > 0.01:
+        #         print(index, output[index], x)
+
+        np.testing.assert_allclose(output, ref_output, rtol=1e-1, atol=1e-1)
+
+
+def gpu_preprocess(tvm_mod):
+    layout_config = relay.transform.LayoutConfig()
+    desired_layouts = {"nn.conv2d": ["NCHW4c", "OIHW4o"]}
+    with layout_config:
+        seq = tvm.transform.Sequential([relay.transform.ConvertLayout(desired_layouts)])
+        with tvm.transform.PassContext(opt_level=3):
+            mod = tvm.IRModule.from_expr(tvm_mod)
+            tvm_mod_nchwc = seq(mod)
+            return tvm_mod_nchwc
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 42, 42)
+    filter_shape = (96, 32, 3, 3)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad_pass():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 40, 40)
+    filter_shape = (96, 32, 2, 2)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(2, 2),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_35_35_strides():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 48, 35, 35)
+    filter_shape = (64, 48, 5, 5)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[2, 2, 2, 2],
+        strides=[1, 1],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(5, 5),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_resnet50_v2_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 224, 224)
+    filter_shape = (64, 3, 7, 7)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 3, 3],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(7, 7),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    # mod, params = relay.testing.init.create_workload(func)
+    np.random.seed(1)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 299, 299)
+    filter_shape = (64, 3, 3, 3)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_1x1_16c16spatial():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 16, 256, 256)
+    filter_shape = (32, 16, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (32, 32, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (4, 32, 4, 4)
+    bias_shape = (1, 4, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=4,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_yolov3_v2_nchw_3c():

Review Comment:
   @csullivan  Looked into CI test results and got an impression that all opencl tests are disabled. It seems we need to enable them in CI but in separate PR



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r871118563


##########
tests/python/relay/test_conv2d_nchw_texture.py:
##########
@@ -0,0 +1,490 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):
+    mod_fp32 = recast(mod, "float32", "float32", ops=["nn.conv2d", "add", "nn.relu"])
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod_fp32, "llvm", params=params1)
+    ctx = tvm.cpu()
+    m = graph_runtime.create(graph, lib, ctx)
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            m.set_input(key, inputs[-1])
+    else:
+        m.set_input("data", inputs[-1])
+    m.set_input(**params)
+    m.run()
+    return [
+        m.get_output(0).asnumpy(),
+    ]
+
+
+# build module run with opencl and cpu, compare results
+def build_run_compare(
+    tvm_mod, params1, input_shape, dtype="float32", target="llvm", gpu_preprocess=None
+):
+
+    rpc_tracker_host = os.environ["TVM_TRACKER_HOST"]
+    rpc_tracker_port = os.environ["TVM_TRACKER_PORT"]
+    if rpc_tracker_host:
+        run_on_host = 0
+        target_host = "llvm -mtriple=arm64-linux-android"
+        rpc_tracker_port = int(rpc_tracker_port)
+    else:
+        run_on_host = 1
+        target_host = "llvm"
+
+    if gpu_preprocess:
+        tvm_mod_nchwc = gpu_preprocess(tvm_mod)
+    else:
+        tvm_mod_nchwc = tvm_mod
+
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(
+            tvm_mod_nchwc, target_host=target_host, target=target, params=params1
+        )
+    if run_on_host:
+        ctx = tvm.opencl()
+        m = graph_runtime.create(graph, lib, ctx)
+    else:
+        from tvm import rpc
+        from tvm.contrib import utils, ndk
+
+        rpc_key = "android"
+        tracker = rpc.connect_tracker(rpc_tracker_host, rpc_tracker_port)
+        remote = tracker.request(rpc_key, priority=0, session_timeout=600)
+        temp = utils.tempdir()
+        dso_binary = "dev_lib_cl.so"
+        dso_binary_path = temp.relpath(dso_binary)
+        ctx = remote.cl(0)
+        lib.export_library(dso_binary_path, ndk.create_shared)
+        remote.upload(dso_binary_path)
+        rlib = remote.load_module(dso_binary)
+        m = graph_runtime.create(graph, rlib, ctx)
+    m.set_input(**params)
+    inputs = []
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            inputs.append(np.random.normal(size=input_shape[key]).astype(dtype))
+            m.set_input(key, inputs[-1])
+    else:
+        inputs.append(np.random.normal(size=input_shape).astype(dtype))
+        m.set_input("data", inputs[-1])
+    m.run()
+
+    ref_outputs = get_reference(tvm_mod, params1, input_shape, inputs)
+    for i, ref_output in enumerate(ref_outputs):
+        tvm_output = m.get_output(i)
+        output = tvm_output.asnumpy()
+        # for index, x in np.ndenumerate(ref_output):
+        #     if abs(output[index] - x) > 0.01:
+        #         print(index, output[index], x)
+
+        np.testing.assert_allclose(output, ref_output, rtol=1e-1, atol=1e-1)
+
+
+def gpu_preprocess(tvm_mod):
+    layout_config = relay.transform.LayoutConfig()
+    desired_layouts = {"nn.conv2d": ["NCHW4c", "OIHW4o"]}
+    with layout_config:
+        seq = tvm.transform.Sequential([relay.transform.ConvertLayout(desired_layouts)])
+        with tvm.transform.PassContext(opt_level=3):
+            mod = tvm.IRModule.from_expr(tvm_mod)
+            tvm_mod_nchwc = seq(mod)
+            return tvm_mod_nchwc
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 42, 42)
+    filter_shape = (96, 32, 3, 3)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad_pass():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 40, 40)
+    filter_shape = (96, 32, 2, 2)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(2, 2),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_35_35_strides():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 48, 35, 35)
+    filter_shape = (64, 48, 5, 5)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[2, 2, 2, 2],
+        strides=[1, 1],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(5, 5),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_resnet50_v2_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 224, 224)
+    filter_shape = (64, 3, 7, 7)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 3, 3],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(7, 7),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    # mod, params = relay.testing.init.create_workload(func)
+    np.random.seed(1)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 299, 299)
+    filter_shape = (64, 3, 3, 3)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_1x1_16c16spatial():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 16, 256, 256)
+    filter_shape = (32, 16, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (32, 32, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (4, 32, 4, 4)
+    bias_shape = (1, 4, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=4,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_yolov3_v2_nchw_3c():

Review Comment:
   I have not verified with nvidia gpu, but they pass successfully on intel integrated graphics and enabled opencl in the platform and tvm. I need to verify if tests run in the CI, but cannot do this due to issues with GPU build in CI



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

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

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


[GitHub] [tvm] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
tests/python/relay/test_conv2d_nhwc_texture.py:
##########
@@ -0,0 +1,651 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):

Review Comment:
   Common utility shared in other test files, consider adding to the utils subdir.



-- 
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] csullivan commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


##########
tests/python/relay/test_conv2d_nchw_texture.py:
##########
@@ -0,0 +1,490 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):
+    mod_fp32 = recast(mod, "float32", "float32", ops=["nn.conv2d", "add", "nn.relu"])
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod_fp32, "llvm", params=params1)
+    ctx = tvm.cpu()
+    m = graph_runtime.create(graph, lib, ctx)
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            m.set_input(key, inputs[-1])
+    else:
+        m.set_input("data", inputs[-1])
+    m.set_input(**params)
+    m.run()
+    return [
+        m.get_output(0).asnumpy(),
+    ]
+
+
+# build module run with opencl and cpu, compare results
+def build_run_compare(
+    tvm_mod, params1, input_shape, dtype="float32", target="llvm", gpu_preprocess=None
+):
+
+    rpc_tracker_host = os.environ["TVM_TRACKER_HOST"]
+    rpc_tracker_port = os.environ["TVM_TRACKER_PORT"]
+    if rpc_tracker_host:
+        run_on_host = 0
+        target_host = "llvm -mtriple=arm64-linux-android"
+        rpc_tracker_port = int(rpc_tracker_port)
+    else:
+        run_on_host = 1
+        target_host = "llvm"
+
+    if gpu_preprocess:
+        tvm_mod_nchwc = gpu_preprocess(tvm_mod)
+    else:
+        tvm_mod_nchwc = tvm_mod
+
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(
+            tvm_mod_nchwc, target_host=target_host, target=target, params=params1
+        )
+    if run_on_host:
+        ctx = tvm.opencl()
+        m = graph_runtime.create(graph, lib, ctx)
+    else:
+        from tvm import rpc
+        from tvm.contrib import utils, ndk
+
+        rpc_key = "android"
+        tracker = rpc.connect_tracker(rpc_tracker_host, rpc_tracker_port)
+        remote = tracker.request(rpc_key, priority=0, session_timeout=600)
+        temp = utils.tempdir()
+        dso_binary = "dev_lib_cl.so"
+        dso_binary_path = temp.relpath(dso_binary)
+        ctx = remote.cl(0)
+        lib.export_library(dso_binary_path, ndk.create_shared)
+        remote.upload(dso_binary_path)
+        rlib = remote.load_module(dso_binary)
+        m = graph_runtime.create(graph, rlib, ctx)
+    m.set_input(**params)
+    inputs = []
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            inputs.append(np.random.normal(size=input_shape[key]).astype(dtype))
+            m.set_input(key, inputs[-1])
+    else:
+        inputs.append(np.random.normal(size=input_shape).astype(dtype))
+        m.set_input("data", inputs[-1])
+    m.run()
+
+    ref_outputs = get_reference(tvm_mod, params1, input_shape, inputs)
+    for i, ref_output in enumerate(ref_outputs):
+        tvm_output = m.get_output(i)
+        output = tvm_output.asnumpy()
+        # for index, x in np.ndenumerate(ref_output):
+        #     if abs(output[index] - x) > 0.01:
+        #         print(index, output[index], x)
+
+        np.testing.assert_allclose(output, ref_output, rtol=1e-1, atol=1e-1)
+
+
+def gpu_preprocess(tvm_mod):
+    layout_config = relay.transform.LayoutConfig()
+    desired_layouts = {"nn.conv2d": ["NCHW4c", "OIHW4o"]}
+    with layout_config:
+        seq = tvm.transform.Sequential([relay.transform.ConvertLayout(desired_layouts)])
+        with tvm.transform.PassContext(opt_level=3):
+            mod = tvm.IRModule.from_expr(tvm_mod)
+            tvm_mod_nchwc = seq(mod)
+            return tvm_mod_nchwc
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 42, 42)
+    filter_shape = (96, 32, 3, 3)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_64x35x35_96x64x3x3_nopad_pass():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 40, 40)
+    filter_shape = (96, 32, 2, 2)
+    bias_shape = (1, 96, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=96,
+        kernel_size=(2, 2),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_35_35_strides():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 48, 35, 35)
+    filter_shape = (64, 48, 5, 5)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[2, 2, 2, 2],
+        strides=[1, 1],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(5, 5),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target, gpu_preprocess)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_resnet50_v2_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 224, 224)
+    filter_shape = (64, 3, 7, 7)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 3, 3],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(7, 7),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    # mod, params = relay.testing.init.create_workload(func)
+    np.random.seed(1)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_inceptionv3_nchw_3c():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 3, 299, 299)
+    filter_shape = (64, 3, 3, 3)
+    bias_shape = (1, 64, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=64,
+        kernel_size=(3, 3),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_1x1_16c16spatial():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 16, 256, 256)
+    filter_shape = (32, 16, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[0, 0, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (32, 32, 4, 4)
+    bias_shape = (1, 32, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=32,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_4x4x4_16c16pad():
+    target = "opencl --device=adreno"
+    dtype = "float16"
+
+    input_shape = (1, 32, 256, 256)
+    filter_shape = (4, 32, 4, 4)
+    bias_shape = (1, 4, 1, 1)
+    A = relay.var("data", shape=input_shape, dtype=dtype)
+    B = relay.var("weight", shape=filter_shape, dtype=dtype)
+    bias = relay.var("bias", shape=bias_shape, dtype=dtype)
+
+    # C = relay.nn.relu(A)
+    conv = relay.nn.conv2d(
+        A,
+        B,
+        data_layout="NCHW",
+        kernel_layout="OIHW",
+        padding=[3, 3, 0, 0],
+        strides=[2, 2],
+        out_dtype=dtype,
+        channels=4,
+        kernel_size=(4, 4),
+    )
+    D = relay.op.add(conv, bias)
+    D = relay.op.nn.relu(D)
+
+    mod = relay.Function([A, B, bias], D)
+    np.random.seed(0)
+    initializer = relay.testing.init.Xavier()
+    filter_data = np.zeros(filter_shape).astype(dtype)
+    bias_data = np.zeros(bias_shape).astype(dtype)
+    initializer("weight", filter_data)
+    initializer("bias", bias_data)
+    params1 = {
+        "weight": tvm.nd.array(filter_data),
+        "bias": tvm.nd.array(bias_data),
+    }
+
+    build_run_compare(mod, params1, {"data": input_shape}, dtype, target)
+
+
+@tvm.testing.requires_opencl
+def test_conv2d_yolov3_v2_nchw_3c():

Review Comment:
   That's accurate, and I agree we can consider enabling them in CI in a separate PR. If you see that these tests pass when running locally and _without_ and RPC tracker that is sufficient. 



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r871372700


##########
src/target/target_kind.cc:
##########
@@ -324,6 +324,7 @@ TVM_REGISTER_TARGET_KIND("opencl", kDLOpenCL)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
     .add_attr_option<Integer>("thread_warp_size", Integer(1))
+    .add_attr_option<Integer>("texture_spatial_limit", Integer(16384))

Review Comment:
   I still do not fully understand the usage model. I left for a while only definition of `texture_spatial_limit` in opencl target and access in python because adding of `kTextureSpatialLimit` in DeviceAttrKind caused a fail during compilation of cuda and as I do not fully understand usage model, don't know how to fix this properly. If I need to extend cuda as well for this constant or just ignore and if ignore in which place `kTextureSpatialLimit` should be 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] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r871368616


##########
tests/python/relay/test_conv2d_nhwc_texture.py:
##########
@@ -0,0 +1,651 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):

Review Comment:
   moved shared functions into utils/adreno_utils.py



##########
tests/python/relay/test_conv2d_nchw_texture.py:
##########
@@ -0,0 +1,490 @@
+# 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 os
+import tvm
+import numpy as np
+from tvm import relay
+from tvm.relay import testing
+from tvm.relay.transform import recast
+from tvm.relay.transform import recast
+from tvm.contrib import graph_runtime
+
+
+def get_reference(mod, params1, input_shape, inputs):
+    mod_fp32 = recast(mod, "float32", "float32", ops=["nn.conv2d", "add", "nn.relu"])
+    with relay.build_config(opt_level=3):
+        graph, lib, params = relay.build(mod_fp32, "llvm", params=params1)
+    ctx = tvm.cpu()
+    m = graph_runtime.create(graph, lib, ctx)
+    if isinstance(input_shape, dict):
+        for key in input_shape:
+            m.set_input(key, inputs[-1])
+    else:
+        m.set_input("data", inputs[-1])
+    m.set_input(**params)
+    m.run()
+    return [
+        m.get_output(0).asnumpy(),
+    ]
+
+
+# build module run with opencl and cpu, compare results
+def build_run_compare(

Review Comment:
   moved shared functions into utils/adreno_utils.py



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r865792149


##########
src/runtime/opencl/opencl_common.h:
##########
@@ -345,6 +345,7 @@ struct BufferDescriptor {
      *         e.g. image2d[height=O, width=IHW]
      */
     kImage2DWeight,
+    kTexture2DNHWC,

Review Comment:
   Done



##########
python/tvm/relay/op/strategy/adreno.py:
##########
@@ -0,0 +1,162 @@
+# 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.
+"""Definition of adreno operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from tvm import topi
+from .generic import *
+from .. import op as _op
+
+
+@conv2d_NCHWc_strategy.register("adreno")
+@conv2d_strategy.register("adreno")
+def conv2d_strategy_adreno(attrs, inputs, out_type, target):
+    """conv2d adreno strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation_h, dilation_w = attrs.get_int_tuple("dilation")
+    groups = attrs.groups
+    data_layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if (data_layout == "NCHW" and kernel_layout == "OIHW") or (
+            data_layout == "NCHW4c" and kernel_layout == "OIHW4o"
+        ):
+            if out_type.dtype == "float16":
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.adreno.conv2d_nchwc),
+                    wrap_topi_schedule(topi.adreno.schedule_conv2d_nchwc),
+                    name="conv2d_nchwc.image2d",
+                    plevel=10,
+                )
+            strategy.add_implementation(
+                wrap_compute_conv2d(topi.adreno.conv2d_nchwc_acc32),
+                wrap_topi_schedule(topi.adreno.schedule_conv2d_nchwc_acc32),
+                name="conv2d_nchwc_tpack.image2d",
+                plevel=20,
+            )
+        elif (data_layout == "NHWC" and kernel_layout == "HWIO") or (
+            data_layout == "NHWC4c" and kernel_layout == "HWIO4o"
+        ):
+            if out_type.dtype == "float16":
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.adreno.conv2d_nhwc),
+                    wrap_topi_schedule(topi.adreno.schedule_conv2d_nhwc),
+                    name="conv2d_nhwc.image2d",
+                    plevel=10,
+                )
+            strategy.add_implementation(
+                wrap_compute_conv2d(topi.adreno.conv2d_nhwc_acc32),
+                wrap_topi_schedule(topi.adreno.schedule_conv2d_nhwc_acc32),
+                name="conv2d_nhwc_acc32.image2d",
+                plevel=20,
+            )
+        else:
+            raise RuntimeError(
+                "Layout not supported: ("
+                + data_layout
+                + ", "
+                + kernel_layout
+                + ") - only support NCHW4c / OIHW4o and NHWC / HWOI layouts for conv2d"
+            )
+    else:
+        # cannot use is_depthwise_conv2d because it does not know about NHWC4c/HWOI4o layouts
+        if data_layout == "NCHW":
+            ic = data.shape[1]
+        elif data_layout == "NCHW4c":
+            ic = data.shape[1] * data.shape[4]
+        elif data_layout == "NHWC":
+            ic = data.shape[3]
+        elif data_layout == "NHWC4c":
+            ic = data.shape[3] * data.shape[4]
+        else:
+            # TODO(amalyshe) add proper error raising

Review Comment:
   Done



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

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

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


[GitHub] [tvm] elvin-n commented on a diff in pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

Posted by GitBox <gi...@apache.org>.
elvin-n commented on code in PR #11161:
URL: https://github.com/apache/tvm/pull/11161#discussion_r864089092


##########
src/runtime/opencl/opencl_common.h:
##########
@@ -345,6 +345,7 @@ struct BufferDescriptor {
      *         e.g. image2d[height=O, width=IHW]
      */
     kImage2DWeight,
+    kTexture2DNHWC,

Review Comment:
   Should we add any AR/TODO into the code?



-- 
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] csullivan merged pull request #11161: Add Adreno GPU target and topi supporting textures with dynamically allocated textures

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


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