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 2020/11/07 09:43:35 UTC

[GitHub] [incubator-tvm] merrymercy opened a new pull request #6877: [AutoScheduler] Add winograd support in tuning networks

merrymercy opened a new pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877


   This PR adds the Winograd support for NHWC layout.
   
   Now it is only enabled for GPU backend, but the CPU backend uses the same compute declaration. I will enable it in later PRs.
   
   cc @jcf94 @FrozenGene @comaniac 


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

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



[GitHub] [incubator-tvm] merrymercy merged pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
merrymercy merged pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877


   


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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877#discussion_r521989104



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -116,8 +116,8 @@ SketchPolicy::SketchPolicy(SearchTask task, CostModel program_cost_model,
   } else if (IsGPUTask(node->search_task)) {
     // Sketch Generation Rules
     node->sketch_rules.push_back(&rule_add_cache_read_stage);
-    node->sketch_rules.push_back(&rule_always_inline);
     node->sketch_rules.push_back(&rule_special_compute_location_gpu);
+    node->sketch_rules.push_back(&rule_always_inline);

Review comment:
       The old order generates wrong sketches for winograd.
   




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

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



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
jcf94 commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877#discussion_r519637879



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -116,8 +116,8 @@ SketchPolicy::SketchPolicy(SearchTask task, CostModel program_cost_model,
   } else if (IsGPUTask(node->search_task)) {
     // Sketch Generation Rules
     node->sketch_rules.push_back(&rule_add_cache_read_stage);
-    node->sketch_rules.push_back(&rule_always_inline);
     node->sketch_rules.push_back(&rule_special_compute_location_gpu);
+    node->sketch_rules.push_back(&rule_always_inline);

Review comment:
       I have the same opinion as Cody here.




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

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



[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877#discussion_r519521607



##########
File path: python/tvm/relay/op/strategy/cuda.py
##########
@@ -206,19 +198,39 @@ def conv2d_strategy_cuda(attrs, inputs, out_type, target):
                         name="conv2d_nhwc_winograd_direct.cuda",
                         plevel=5,
                     )
-            if target.kind.name == "cuda":
-                if nvcc.have_tensorcore(tvm.gpu(0).compute_version):
-                    if (
-                        (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
-                        or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
-                        or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
-                    ):
-                        strategy.add_implementation(
-                            wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
-                            wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
-                            name="conv2d_nhwc_tensorcore.cuda",
-                            plevel=20,
-                        )
+            if (
+                target.kind.name == "cuda"
+                and nvcc.have_tensorcore(tvm.gpu(0).compute_version)
+                and (
+                    (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
+                    or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
+                    or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
+                )
+            ):
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
+                    wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
+                    name="conv2d_nhwc_tensorcore.cuda",
+                    plevel=20,
+                )
+
+            # register auto-scheduler implementations
+            judge_winograd_auto_scheduler = (
+                ("float" in data.dtype and "float" in kernel.dtype)

Review comment:
       float16 should work too.

##########
File path: python/tvm/topi/nn/conv2d.py
##########
@@ -943,3 +943,212 @@ def unpack_NCHWc_to_nchw(packed_out, out_dtype):
         tag=tag.INJECTIVE + ",unpack_nchwc",
     )
     return unpacked_out
+
+
+def _conv2d_winograd_nhwc_impl(
+    data,
+    weight,
+    strides,
+    padding,
+    dilation,
+    out_dtype,
+    tile_size,
+    pre_computed=False,
+):
+    """Conv2D Winograd implementation in NHWC layout.
+    This is a clean version to be used by the auto-scheduler for both CPU and GPU.
+
+    Parameters
+    ----------
+    data : tvm.Tensor
+        4-D with shape [batch, in_height, in_width, in_channel]
+    weight : tvm.Tensor
+        4-D with shape [filter_height, filter_width, in_channel, num_filter]
+    strides : int or a list/tuple of two ints
+        stride size, or [stride_height, stride_width]
+    padding : int or a list/tuple of two ints
+        padding size, or [pad_height, pad_width]
+    dilation: int or a list/tuple of two ints
+        dilation size, or [dilation_height, dilation_width]
+    out_dtype : str, optional
+        Specifies the output data type.
+    tile_size : int
+        The size of the tile to use for the Winograd filter
+    pre_computed: bool
+        Whether the kernel is precomputed
+
+    Returns
+    -------
+    output : tvm.Tensor
+        4-D with shape [batch, out_height, out_width, out_channel]
+    """
+    N, H, W, CI = get_const_tuple(data.shape)
+    if isinstance(dilation, int):
+        dilation_h = dilation_w = dilation
+    else:
+        dilation_h, dilation_w = dilation
+
+    assert (dilation_h, dilation_w) == (1, 1), "Does not support dilation"
+    if not pre_computed:
+        KH, KW, CI, CO = get_const_tuple(weight.shape)
+    else:
+        H_CAT, W_CAT, CO, CI = get_const_tuple(weight.shape)
+        KH, KW = H_CAT - tile_size + 1, W_CAT - tile_size + 1
+
+    pad_t, pad_l, pad_b, pad_r = get_pad_tuple(padding, (KH, KW))
+    HSTR, WSTR = (strides, strides) if isinstance(strides, int) else strides
+    assert HSTR == 1 and WSTR == 1 and KH == 3 and KW == 3
+
+    r = KW
+    m = tile_size
+    alpha = m + r - 1
+    A, B, G = winograd_transform_matrices(m, r, out_dtype)
+
+    H = (H + pad_t + pad_b - KH) // HSTR + 1
+    W = (W + pad_l + pad_r - KW) // WSTR + 1
+    nH, nW = (H + m - 1) // m, (W + m - 1) // m
+    P = N * nH * nW
+
+    pad_extra = (nW - 1) * m + alpha - (H + pad_t + pad_b)
+    data_pad = pad(
+        data, (0, pad_t, pad_l, 0), (0, pad_b + pad_extra, pad_r + pad_extra, 0), name="data_pad"
+    )
+
+    if not pre_computed:
+        r_kh = te.reduce_axis((0, KH), name="r_kh")
+        r_kw = te.reduce_axis((0, KW), name="r_kw")
+        kernel_pack = te.compute(
+            (alpha, alpha, CO, CI),
+            lambda eps, nu, co, ci: te.sum(
+                weight[r_kh][r_kw][ci][co] * G[eps][r_kh] * G[nu][r_kw], axis=[r_kh, r_kw]
+            ),
+            name="kernel_pack",
+        )
+    else:
+        kernel_pack = weight
+
+    # pack data tile
+    input_tile = te.compute(
+        (alpha, alpha, P, CI),
+        lambda eps, nu, p, ci: data_pad[p // (nH * nW)][((p // nW) % nH) * m + eps][
+            (p % nW) * m + nu
+        ][ci],
+        name="input_tile",
+    )
+
+    # transform data
+    r_a = te.reduce_axis((0, alpha), "r_a")
+    r_b = te.reduce_axis((0, alpha), "r_b")
+    data_pack = te.compute(
+        (alpha, alpha, P, CI),
+        lambda eps, nu, p, ci: te.sum(
+            input_tile[r_a][r_b][p][ci] * B[r_a][eps] * B[r_b][nu], axis=[r_a, r_b]
+        ),
+        name="data_pack",
+        attrs={"auto_scheduler_simplify_const_tensor_indices": ["eps", "nu", "r_a", "r_b"]},

Review comment:
       We could add one comment what the attribute is to do and why we need this for auto scheduler




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877#discussion_r519530910



##########
File path: python/tvm/relay/op/strategy/cuda.py
##########
@@ -206,19 +198,39 @@ def conv2d_strategy_cuda(attrs, inputs, out_type, target):
                         name="conv2d_nhwc_winograd_direct.cuda",
                         plevel=5,
                     )
-            if target.kind.name == "cuda":
-                if nvcc.have_tensorcore(tvm.gpu(0).compute_version):
-                    if (
-                        (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
-                        or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
-                        or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
-                    ):
-                        strategy.add_implementation(
-                            wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
-                            wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
-                            name="conv2d_nhwc_tensorcore.cuda",
-                            plevel=20,
-                        )
+            if (
+                target.kind.name == "cuda"
+                and nvcc.have_tensorcore(tvm.gpu(0).compute_version)
+                and (
+                    (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
+                    or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
+                    or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
+                )
+            ):
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
+                    wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
+                    name="conv2d_nhwc_tensorcore.cuda",
+                    plevel=20,
+                )
+
+            # register auto-scheduler implementations
+            judge_winograd_auto_scheduler = (
+                ("float" in data.dtype and "float" in kernel.dtype)

Review comment:
       nvm, I was thinking in a wrong way.




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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877#discussion_r521989104



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -116,8 +116,8 @@ SketchPolicy::SketchPolicy(SearchTask task, CostModel program_cost_model,
   } else if (IsGPUTask(node->search_task)) {
     // Sketch Generation Rules
     node->sketch_rules.push_back(&rule_add_cache_read_stage);
-    node->sketch_rules.push_back(&rule_always_inline);
     node->sketch_rules.push_back(&rule_special_compute_location_gpu);
+    node->sketch_rules.push_back(&rule_always_inline);

Review comment:
       The old order generates wrong sketches for winograd.
   For some stages, we have to apply `rule_special_compute_location_gpu`. But the older order will just inline them.




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6877: [AutoScheduler] Add winograd support in tuning networks

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-tvm/pull/6877#discussion_r519495596



##########
File path: src/auto_scheduler/search_policy/sketch_policy.cc
##########
@@ -116,8 +116,8 @@ SketchPolicy::SketchPolicy(SearchTask task, CostModel program_cost_model,
   } else if (IsGPUTask(node->search_task)) {
     // Sketch Generation Rules
     node->sketch_rules.push_back(&rule_add_cache_read_stage);
-    node->sketch_rules.push_back(&rule_always_inline);
     node->sketch_rules.push_back(&rule_special_compute_location_gpu);
+    node->sketch_rules.push_back(&rule_always_inline);

Review comment:
       Out of curiosity, why changed to this order? I saw that the condition of applying special compute location also checks should be always inlined, so then it should be fine to keep always inline to go first?

##########
File path: python/tvm/relay/op/strategy/cuda.py
##########
@@ -206,19 +198,39 @@ def conv2d_strategy_cuda(attrs, inputs, out_type, target):
                         name="conv2d_nhwc_winograd_direct.cuda",
                         plevel=5,
                     )
-            if target.kind.name == "cuda":
-                if nvcc.have_tensorcore(tvm.gpu(0).compute_version):
-                    if (
-                        (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
-                        or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
-                        or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
-                    ):
-                        strategy.add_implementation(
-                            wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
-                            wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
-                            name="conv2d_nhwc_tensorcore.cuda",
-                            plevel=20,
-                        )
+            if (
+                target.kind.name == "cuda"
+                and nvcc.have_tensorcore(tvm.gpu(0).compute_version)
+                and (
+                    (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
+                    or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
+                    or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
+                )
+            ):
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
+                    wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
+                    name="conv2d_nhwc_tensorcore.cuda",
+                    plevel=20,
+                )
+
+            # register auto-scheduler implementations
+            judge_winograd_auto_scheduler = (

Review comment:
       Following the above style of judging Winograd, we can remove this variable.

##########
File path: python/tvm/relay/op/strategy/cuda.py
##########
@@ -206,19 +198,39 @@ def conv2d_strategy_cuda(attrs, inputs, out_type, target):
                         name="conv2d_nhwc_winograd_direct.cuda",
                         plevel=5,
                     )
-            if target.kind.name == "cuda":
-                if nvcc.have_tensorcore(tvm.gpu(0).compute_version):
-                    if (
-                        (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
-                        or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
-                        or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
-                    ):
-                        strategy.add_implementation(
-                            wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
-                            wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
-                            name="conv2d_nhwc_tensorcore.cuda",
-                            plevel=20,
-                        )
+            if (
+                target.kind.name == "cuda"
+                and nvcc.have_tensorcore(tvm.gpu(0).compute_version)
+                and (
+                    (N % 16 == 0 and CI % 16 == 0 and CO % 16 == 0)
+                    or (N % 8 == 0 and CI % 16 == 0 and CO % 32 == 0)
+                    or (N % 32 == 0 and CI % 16 == 0 and CO % 8 == 0)
+                )
+            ):
+                strategy.add_implementation(
+                    wrap_compute_conv2d(topi.cuda.conv2d_nhwc_tensorcore),
+                    wrap_topi_schedule(topi.cuda.schedule_conv2d_nhwc_tensorcore),
+                    name="conv2d_nhwc_tensorcore.cuda",
+                    plevel=20,
+                )
+
+            # register auto-scheduler implementations
+            judge_winograd_auto_scheduler = (
+                ("float" in data.dtype and "float" in kernel.dtype)

Review comment:
       Will `dtype` be "float32"?




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

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