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/05/27 14:12:11 UTC

[GitHub] [tvm] quic-sanirudh opened a new pull request, #11489: [Topi] [Hexagon] Conv2d slice op initial version

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

   This patch adds the initial python implementation conv2d slice op for hexagon.
   
   Slice ops are basically ops that make certain assumptions about the input and output dimensions and are expected to be called after the original op has been sliced according to those dimensions at the graph level.


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

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

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


[GitHub] [tvm] quic-sanirudh commented on a diff in pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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


##########
python/tvm/topi/hexagon/slice_ops/conv2d.py:
##########
@@ -0,0 +1,223 @@
+# 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=line-too-long
+
+"""Hexagon slice conv2d compute and schedule"""
+import typing
+
+import tvm
+from tvm import te
+from ..utils import get_layout_transform_fn
+
+
+def conv2d_compute(
+    activations: te.Tensor,
+    weights: te.Tensor,
+    out_shape: typing.Tuple,
+    stride: typing.Tuple,
+    dilation: typing.Tuple,
+    dtype: str,
+    output_name: str,
+) -> te.Tensor:
+    """Compute for slice conv2d op for hexagon.
+
+    This op makes the following assumptions:
+    1. This op is written for a sliced convolution with 2d physical buffers
+    2. The input activations is assumed to be in NHWC layout and filter is in HWIO layout
+    3. Grouped convolutions are not supported. and there will be a separate compute definition for depthwise convolution
+    4. In order to get grouped convolutions, it is assumed that the op will be sliced according to the groups and multiple calls to this compute would be placed.
+
+    Parameters
+    ----------
+    activations : te.Tensor
+        Input activations padded for inner dimension size
+    weights : te.Tensor
+        Weights without dilation
+    out_shape : typing.Tuple
+        The logical output shape without considering input padding
+    stride : typing.Tuple
+        stride
+    dilation : typing.Tuple
+        dilation
+    dtype : str
+        dtype
+    output_name : str
+        The name to be given to output. This would become the block name for the corresponding STIR compute
+
+    Returns
+    -------
+    output : te.Tensor
+        Output of applying 2D convolution of Weights on Input
+    """
+
+    filt_shape = weights.shape
+
+    reduce_channel = tvm.te.reduce_axis((0, filt_shape[2]), name="reduce_channel")
+    reduce_height = tvm.te.reduce_axis((0, filt_shape[0]), name="reduce_height")
+    reduce_width = tvm.te.reduce_axis((0, filt_shape[1]), name="reduce_width")
+    stride_height, stride_width = stride
+    dilation_height, dilation_width = dilation
+    output = tvm.te.compute(
+        out_shape,
+        lambda n, h, w, c: tvm.te.sum(
+            (
+                activations[
+                    n,
+                    h * stride_height + reduce_height * dilation_height,
+                    w * stride_width + reduce_width * dilation_width,
+                    reduce_channel,
+                ]
+                * weights[reduce_height, reduce_width, reduce_channel, c]
+            ).astype(dtype),
+            axis=[reduce_channel, reduce_height, reduce_width],
+        ),
+        name=output_name,
+    )
+    return output
+
+
+def conv2d_te_schedule(
+    out: te.Tensor,
+    ins: typing.List[te.Tensor],
+    transform_activation_layout: str,

Review Comment:
   Yes that does seem quite odd, thanks for the catch. I've updated the code to use `str` based layout for weights as well.



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

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

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


[GitHub] [tvm] quic-sanirudh commented on pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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

   > I restarted the job, not sure why CI wasn't started.
   
   Thanks for this @mehrdadh. I just saw that there was a conflict in a file, so I just rebased my branch on top of main.


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

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

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


[GitHub] [tvm] quic-sanirudh commented on pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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

   @Lunderberg @mehrdadh Are there any more comments and if not, could you please help get this 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] Lunderberg commented on a diff in pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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


##########
python/tvm/topi/hexagon/slice_ops/conv2d.py:
##########
@@ -0,0 +1,223 @@
+# 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=line-too-long
+
+"""Hexagon slice conv2d compute and schedule"""
+import typing
+
+import tvm
+from tvm import te
+from ..utils import get_layout_transform_fn
+
+
+def conv2d_compute(
+    activations: te.Tensor,
+    weights: te.Tensor,
+    out_shape: typing.Tuple,
+    stride: typing.Tuple,
+    dilation: typing.Tuple,
+    dtype: str,
+    output_name: str,
+) -> te.Tensor:
+    """Compute for slice conv2d op for hexagon.
+
+    This op makes the following assumptions:
+    1. This op is written for a sliced convolution with 2d physical buffers
+    2. The input activations is assumed to be in NHWC layout and filter is in HWIO layout
+    3. Grouped convolutions are not supported. and there will be a separate compute definition for depthwise convolution
+    4. In order to get grouped convolutions, it is assumed that the op will be sliced according to the groups and multiple calls to this compute would be placed.
+
+    Parameters
+    ----------
+    activations : te.Tensor
+        Input activations padded for inner dimension size
+    weights : te.Tensor
+        Weights without dilation
+    out_shape : typing.Tuple
+        The logical output shape without considering input padding
+    stride : typing.Tuple
+        stride
+    dilation : typing.Tuple
+        dilation
+    dtype : str
+        dtype
+    output_name : str
+        The name to be given to output. This would become the block name for the corresponding STIR compute
+
+    Returns
+    -------
+    output : te.Tensor
+        Output of applying 2D convolution of Weights on Input
+    """
+
+    filt_shape = weights.shape
+
+    reduce_channel = tvm.te.reduce_axis((0, filt_shape[2]), name="reduce_channel")
+    reduce_height = tvm.te.reduce_axis((0, filt_shape[0]), name="reduce_height")
+    reduce_width = tvm.te.reduce_axis((0, filt_shape[1]), name="reduce_width")
+    stride_height, stride_width = stride
+    dilation_height, dilation_width = dilation
+    output = tvm.te.compute(
+        out_shape,
+        lambda n, h, w, c: tvm.te.sum(
+            (
+                activations[
+                    n,
+                    h * stride_height + reduce_height * dilation_height,
+                    w * stride_width + reduce_width * dilation_width,
+                    reduce_channel,
+                ]
+                * weights[reduce_height, reduce_width, reduce_channel, c]
+            ).astype(dtype),
+            axis=[reduce_channel, reduce_height, reduce_width],
+        ),
+        name=output_name,
+    )
+    return output
+
+
+def conv2d_te_schedule(
+    out: te.Tensor,
+    ins: typing.List[te.Tensor],
+    transform_activation_layout: str,

Review Comment:
   Thank you on the change, and LGTM!



-- 
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] Lunderberg commented on a diff in pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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


##########
python/tvm/topi/hexagon/slice_ops/conv2d.py:
##########
@@ -0,0 +1,223 @@
+# 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=line-too-long
+
+"""Hexagon slice conv2d compute and schedule"""
+import typing
+
+import tvm
+from tvm import te
+from ..utils import get_layout_transform_fn
+
+
+def conv2d_compute(
+    activations: te.Tensor,
+    weights: te.Tensor,
+    out_shape: typing.Tuple,
+    stride: typing.Tuple,
+    dilation: typing.Tuple,
+    dtype: str,
+    output_name: str,
+) -> te.Tensor:
+    """Compute for slice conv2d op for hexagon.
+
+    This op makes the following assumptions:
+    1. This op is written for a sliced convolution with 2d physical buffers
+    2. The input activations is assumed to be in NHWC layout and filter is in HWIO layout
+    3. Grouped convolutions are not supported. and there will be a separate compute definition for depthwise convolution
+    4. In order to get grouped convolutions, it is assumed that the op will be sliced according to the groups and multiple calls to this compute would be placed.
+
+    Parameters
+    ----------
+    activations : te.Tensor
+        Input activations padded for inner dimension size
+    weights : te.Tensor
+        Weights without dilation
+    out_shape : typing.Tuple
+        The logical output shape without considering input padding
+    stride : typing.Tuple
+        stride
+    dilation : typing.Tuple
+        dilation
+    dtype : str
+        dtype
+    output_name : str
+        The name to be given to output. This would become the block name for the corresponding STIR compute
+
+    Returns
+    -------
+    output : te.Tensor
+        Output of applying 2D convolution of Weights on Input
+    """
+
+    filt_shape = weights.shape
+
+    reduce_channel = tvm.te.reduce_axis((0, filt_shape[2]), name="reduce_channel")
+    reduce_height = tvm.te.reduce_axis((0, filt_shape[0]), name="reduce_height")
+    reduce_width = tvm.te.reduce_axis((0, filt_shape[1]), name="reduce_width")
+    stride_height, stride_width = stride
+    dilation_height, dilation_width = dilation
+    output = tvm.te.compute(
+        out_shape,
+        lambda n, h, w, c: tvm.te.sum(
+            (
+                activations[
+                    n,
+                    h * stride_height + reduce_height * dilation_height,
+                    w * stride_width + reduce_width * dilation_width,
+                    reduce_channel,
+                ]
+                * weights[reduce_height, reduce_width, reduce_channel, c]
+            ).astype(dtype),
+            axis=[reduce_channel, reduce_height, reduce_width],
+        ),
+        name=output_name,
+    )
+    return output
+
+
+def conv2d_te_schedule(
+    out: te.Tensor,
+    ins: typing.List[te.Tensor],
+    transform_activation_layout: str,

Review Comment:
   This seems a bit odd to have one layout as a string and another as a callable.  Can we use the same convention for both?



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

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

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


[GitHub] [tvm] mehrdadh commented on pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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

   I restarted the job, not sure why CI wasn't started.


-- 
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] kparzysz-quic merged pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

Posted by GitBox <gi...@apache.org>.
kparzysz-quic merged PR #11489:
URL: https://github.com/apache/tvm/pull/11489


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

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

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


[GitHub] [tvm] quic-sanirudh commented on pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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

   @Lunderberg @cconvey Could you please take a look at this patch when you get a chance. I just rebased it to take advantage of Jyotsna's utils functions, so I think it's ready for review now.


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

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

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


[GitHub] [tvm] mehrdadh commented on pull request #11489: [Topi] [Hexagon] Conv2d slice op initial version

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

   cc @Lunderberg @cconvey for review.


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

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

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