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/04/08 03:28:16 UTC

[GitHub] [incubator-tvm] masahi opened a new pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

masahi opened a new pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272
 
 
   This is a reimplementation of #4741 based on the new annotator support for composite added in #5261. This is the first real use case of composite in the code base. It is purely for demonstration purpose and not intended to be used for performance critical scenarios. 
   
   Due to the manually inlined tensors in c codegen, full mobilenet execution is disabled in the test. I tried but it took more than 5GB of RAM and compilation didn't finish in a reasonable time. The related issue discussed in https://discuss.tvm.ai/t/external-codegen-constant-tensors-in-c-codegen/5890/

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405842433
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -30,14 +30,108 @@
 #include <tvm/runtime/registry.h>
 
 #include <fstream>
+#include <numeric>
 #include <sstream>
 
+#include "../../utils.h"
 #include "../codegen_c/codegen_c.h"
 
 namespace tvm {
 namespace relay {
 namespace contrib {
 
+using namespace backend;
+
+const CallNode* GetRootConv2DCall(const CallNode* relu_call) {
+  CHECK(relu_call && IsOp(relu_call, "nn.relu"));
+  const auto relu_arg = relu_call->args[0];
+  const CallNode* add_call = relu_arg.as<CallNode>();
+  CHECK(add_call && IsOp(add_call, "add"));
+  const auto add_arg = add_call->args[0];
+  const CallNode* conv_call = add_arg.as<CallNode>();
+  CHECK(conv_call && IsOp(conv_call, "nn.conv2d"));
+  return conv_call;
+}
 
 Review comment:
   Thanks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611269825
 
 
   Please try to retrigger https://github.com/apache/incubator-tvm/issues/5285

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612238775
 
 
   @masahi please rebase when you get a chance 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611248042
 
 
   @tqchen Can you have a look at CI problem? https://ci.tvm.ai/blue/organizations/jenkins/tvm/detail/PR-5272/7/pipeline/152

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405785985
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   Yep, that looks like a bug :) I'll take a quick look now, but I'll probably have a fix PR up tomorrow.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-610732150
 
 
   Please review @zhiics @mbaret @comaniac @trevor-m 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
comaniac commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612258124
 
 
   > @zhiics @comaniac The change to the vm/compiler.cc that moved second fusion before Inline has been reverted in #5277. Should I bring it back in this PR?
   
   Hey sorry for that. Yes that was a mistake. When I was rebasing to the master, the commit that decouples DNNL and VM part was reapplied, so #5288 was reverted as you pointed out. If possible, would you cherry-pick it back in this PR? Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405827170
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -133,83 +209,100 @@ class CodegenDNNL : public ExprVisitor, public CodegenCBase {
   }
 
  private:
-  std::vector<std::string> Conv2d(const CallNode* call) {
-    std::vector<std::string> args;
-    const auto* conv2d_attr = call->attrs.as<Conv2DAttrs>();
-    CHECK(conv2d_attr);
-
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
+  struct GenerateBodyOutput {
+    std::string decl, buf;
+    int out_size = 1;
+    std::string out;
+  };
 
-    // Args: N, C, H, W
-    for (auto s : ishape) {
-      args.push_back(std::to_string(s));
+  std::vector<std::string> GetArgumentNames(const CallNode* call) {
+    std::vector<std::string> arg_names;
+    for (size_t i = 0; i < call->args.size(); ++i) {
+      VisitExpr(call->args[i]);
+      for (auto out : out_) {
+        arg_names.push_back(out.name);
+      }
     }
-
-    // Args: O, G, Ph, Pw, Kh, Kw, Sh, Sw
-    args.push_back(std::to_string(wshape[0]));
-    args.push_back(std::to_string(conv2d_attr->groups));
-    args.push_back(std::to_string(conv2d_attr->padding[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->padding[1].as<IntImmNode>()->value));
-    args.push_back(std::to_string(wshape[2]));
-    args.push_back(std::to_string(wshape[3]));
-    args.push_back(std::to_string(conv2d_attr->strides[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->strides[1].as<IntImmNode>()->value));
-
-    return args;
+    return arg_names;
   }
 
-  std::vector<std::string> Dense(const CallNode* call) {
-    std::vector<std::string> args;
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
-
-    // Args: N, C, O
-    args.push_back(std::to_string(ishape[0]));
-    args.push_back(std::to_string(ishape[1]));
-    args.push_back(std::to_string(wshape[0]));
+  GenerateBodyOutput GenerateOpCall(const CallNode* call) {
+    const auto* op_node = call->op.as<OpNode>();
+    CHECK(op_node) << "OpNode expected, got something else";
 
 Review comment:
   yes, my bad.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406382408
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,128 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_dnnl_fuse():
+    def make_pattern(with_bias=True):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        if with_bias:
+            conv_out = relay.add(conv, bias)
 
 Review comment:
   Should this be ```relay.nn.bias_add```? On my side, I had ```nn.bias_add``` after conv2d and noticed that the GetRootCall would fail unless I changed ```add``` to ```nn.bias_add``` in the expected op list.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405766534
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   @alexwong Could you give some more detail on the issue you're facing? I'd be happy to 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405827170
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -133,83 +209,100 @@ class CodegenDNNL : public ExprVisitor, public CodegenCBase {
   }
 
  private:
-  std::vector<std::string> Conv2d(const CallNode* call) {
-    std::vector<std::string> args;
-    const auto* conv2d_attr = call->attrs.as<Conv2DAttrs>();
-    CHECK(conv2d_attr);
-
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
+  struct GenerateBodyOutput {
+    std::string decl, buf;
+    int out_size = 1;
+    std::string out;
+  };
 
-    // Args: N, C, H, W
-    for (auto s : ishape) {
-      args.push_back(std::to_string(s));
+  std::vector<std::string> GetArgumentNames(const CallNode* call) {
+    std::vector<std::string> arg_names;
+    for (size_t i = 0; i < call->args.size(); ++i) {
+      VisitExpr(call->args[i]);
+      for (auto out : out_) {
+        arg_names.push_back(out.name);
+      }
     }
-
-    // Args: O, G, Ph, Pw, Kh, Kw, Sh, Sw
-    args.push_back(std::to_string(wshape[0]));
-    args.push_back(std::to_string(conv2d_attr->groups));
-    args.push_back(std::to_string(conv2d_attr->padding[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->padding[1].as<IntImmNode>()->value));
-    args.push_back(std::to_string(wshape[2]));
-    args.push_back(std::to_string(wshape[3]));
-    args.push_back(std::to_string(conv2d_attr->strides[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->strides[1].as<IntImmNode>()->value));
-
-    return args;
+    return arg_names;
   }
 
-  std::vector<std::string> Dense(const CallNode* call) {
-    std::vector<std::string> args;
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
-
-    // Args: N, C, O
-    args.push_back(std::to_string(ishape[0]));
-    args.push_back(std::to_string(ishape[1]));
-    args.push_back(std::to_string(wshape[0]));
+  GenerateBodyOutput GenerateOpCall(const CallNode* call) {
+    const auto* op_node = call->op.as<OpNode>();
+    CHECK(op_node) << "OpNode expected, got something else";
 
 Review comment:
   yes, my bad.
   
   Update: I meant, call->op->GetTypekey()

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612254537
 
 
   @zhiics @comaniac The change to the vm/compiler.cc that moved second fusion before Inline has been reverted in #5277. Should I bring it back in this 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612313039
 
 
   Thanks @zhiics @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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612259904
 
 
   @comaniac Done. Please take a look at the last commit

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612242732
 
 
   Yes I'm working on fixing it. Resolving conflict turned out a bit complicated, so I wanted to start fresh.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405817348
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -30,14 +30,108 @@
 #include <tvm/runtime/registry.h>
 
 #include <fstream>
+#include <numeric>
 #include <sstream>
 
+#include "../../utils.h"
 #include "../codegen_c/codegen_c.h"
 
 namespace tvm {
 namespace relay {
 namespace contrib {
 
+using namespace backend;
+
+const CallNode* GetRootConv2DCall(const CallNode* relu_call) {
+  CHECK(relu_call && IsOp(relu_call, "nn.relu"));
+  const auto relu_arg = relu_call->args[0];
+  const CallNode* add_call = relu_arg.as<CallNode>();
+  CHECK(add_call && IsOp(add_call, "add"));
+  const auto add_arg = add_call->args[0];
+  const CallNode* conv_call = add_arg.as<CallNode>();
+  CHECK(conv_call && IsOp(conv_call, "nn.conv2d"));
+  return conv_call;
+}
 
 Review comment:
   yes exactly. I'm working on improving it right now, to support conv2d + relu pattern. I'm adding "depth" and "expected_op_names" arguments.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405770539
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   You can't. Part of the benefit of mergecomposite comes from wrapping the patterns in an easy-to-detect function.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405833302
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -30,14 +30,108 @@
 #include <tvm/runtime/registry.h>
 
 #include <fstream>
+#include <numeric>
 #include <sstream>
 
+#include "../../utils.h"
 #include "../codegen_c/codegen_c.h"
 
 namespace tvm {
 namespace relay {
 namespace contrib {
 
+using namespace backend;
+
+const CallNode* GetRootConv2DCall(const CallNode* relu_call) {
+  CHECK(relu_call && IsOp(relu_call, "nn.relu"));
+  const auto relu_arg = relu_call->args[0];
+  const CallNode* add_call = relu_arg.as<CallNode>();
+  CHECK(add_call && IsOp(add_call, "add"));
+  const auto add_arg = add_call->args[0];
+  const CallNode* conv_call = add_arg.as<CallNode>();
+  CHECK(conv_call && IsOp(conv_call, "nn.conv2d"));
+  return conv_call;
+}
 
 Review comment:
   Sounds good to me. Backend I'm working on can use this, so it would be nice if this can move to utils.h where you moved IsOp and GetShape.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405826461
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -133,83 +209,100 @@ class CodegenDNNL : public ExprVisitor, public CodegenCBase {
   }
 
  private:
-  std::vector<std::string> Conv2d(const CallNode* call) {
-    std::vector<std::string> args;
-    const auto* conv2d_attr = call->attrs.as<Conv2DAttrs>();
-    CHECK(conv2d_attr);
-
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
+  struct GenerateBodyOutput {
+    std::string decl, buf;
+    int out_size = 1;
+    std::string out;
+  };
 
-    // Args: N, C, H, W
-    for (auto s : ishape) {
-      args.push_back(std::to_string(s));
+  std::vector<std::string> GetArgumentNames(const CallNode* call) {
+    std::vector<std::string> arg_names;
+    for (size_t i = 0; i < call->args.size(); ++i) {
+      VisitExpr(call->args[i]);
+      for (auto out : out_) {
+        arg_names.push_back(out.name);
+      }
     }
-
-    // Args: O, G, Ph, Pw, Kh, Kw, Sh, Sw
-    args.push_back(std::to_string(wshape[0]));
-    args.push_back(std::to_string(conv2d_attr->groups));
-    args.push_back(std::to_string(conv2d_attr->padding[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->padding[1].as<IntImmNode>()->value));
-    args.push_back(std::to_string(wshape[2]));
-    args.push_back(std::to_string(wshape[3]));
-    args.push_back(std::to_string(conv2d_attr->strides[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->strides[1].as<IntImmNode>()->value));
-
-    return args;
+    return arg_names;
   }
 
-  std::vector<std::string> Dense(const CallNode* call) {
-    std::vector<std::string> args;
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
-
-    // Args: N, C, O
-    args.push_back(std::to_string(ishape[0]));
-    args.push_back(std::to_string(ishape[1]));
-    args.push_back(std::to_string(wshape[0]));
+  GenerateBodyOutput GenerateOpCall(const CallNode* call) {
+    const auto* op_node = call->op.as<OpNode>();
+    CHECK(op_node) << "OpNode expected, got something else";
 
 Review comment:
   You mean `call->GetTypeKey()` right? When this check fails, op_node is nullptr.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406401569
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   Hey I just tried your case with #5277 and it works fine. Maybe we can push to merge that PR so that this problem will be gone naturally.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406487275
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,128 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_dnnl_fuse():
+    def make_pattern(with_bias=True):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        if with_bias:
+            conv_out = relay.add(conv, bias)
 
 Review comment:
   No, since I use this pattern to detect conv + add + relu that come from decomposing the batch norm.  `SimplyfyInference` generates `relay.add`, see https://github.com/apache/incubator-tvm/blob/14ba49c60c49474a564f990363de9d114c9b019b/src/relay/transforms/simplify_inference.cc#L54
   
   We can have both conv2d + add + relu and conv2d + bias_add + relu patterns in the table.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612260657
 
 
   Changes to multi output support in dnnl/codegen.cc has also been reverted in #5277, but I've already integrated that change in this PR last night.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405815328
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
 
 Review comment:
   Yes, some common patterns could be moved to dnnl.py. Let's keep it here for now and have a separate PR to do that.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611273441
 
 
   Yes I got that error too, but also there is a problem on i386 CI causing segfault from crt. https://ci.tvm.ai/blue/organizations/jenkins/tvm/detail/PR-5272/7/pipeline/152. This PR https://github.com/apache/incubator-tvm/pull/5280 also had the same segfault from i386.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405759315
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   I'm just curious but is it possible to try MergeCompilerRegions here? In my own testing, I seem to be having issues with composite functions and that pass and I think in most use-cases, we would want to use that pass when going through this flow.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406404843
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   Looks like that is caused by wrongly adding the `default` compiler annotations. Let's work on reviewing #5277 and merge these bunch of fixes. 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405855030
 
 

 ##########
 File path: src/relay/backend/utils.h
 ##########
 @@ -152,8 +152,35 @@ inline bool IsOp(const CallNode* call, const std::string& op_name) {
   Op op = GetRef<Op>(op_node);
   return op == Op::Get(op_name);
 }
+
+/*!
+ * \brief Retrieve the "root" conv2d op nested inside a fused call, such as conv2d + relu.
+ * \param call A Relay call node. Typically nn.relu when called the first time.
+ * \param depth The number of calls before conv2d call, counting from current_call.
+ * \param expected_op_names The names of ops in this fused call. Example: {"nn.conv2d", "add",
+ * "nn.relu"}
+ * \return conv2d op at the root
+ */
+
+inline const CallNode* GetRootConv2DCall(const CallNode* current_call, int depth,
+                                         const std::vector<std::string>& expected_op_names) {
+  CHECK(current_call && depth >= 0);
+
+  if (depth == 0) {
+    CHECK(IsOp(current_call, "nn.conv2d"));
+    return current_call;
+  }
+
+  CHECK(depth < expected_op_names.size() && IsOp(current_call, expected_op_names[depth]));
+  CHECK_GT(current_call->args.size(), 0);
+
+  const auto* next_call = current_call->args[0].as<CallNode>();
+  return GetRootConv2DCall(next_call, depth - 1, expected_op_names);
+}
 
 Review comment:
   Renamed to `"GetRootCall` and removed the check of `nn.conv2d` so that it can be used for other root ops. `expected_op_names` is required. Please have 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics edited a comment on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics edited a comment on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612242010
 
 
   @masahi just a reminder, probably you didn't rebase again the master, other changes are here 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406382408
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,128 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_dnnl_fuse():
+    def make_pattern(with_bias=True):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        if with_bias:
+            conv_out = relay.add(conv, bias)
 
 Review comment:
   Should this be ```relay.nn.bias_add```? On my side, I had ```nn.bias_add``` after conv2d and noticed that the GetRootCall would fail unless I changed ```add``` to ```nn.bias_add```.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405771851
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   Thanks. @alexwong So the answer is both approaches are possible, but composite is more general and it also enables easily detecting fused ops inside codegen, like I do in this line https://github.com/apache/incubator-tvm/pull/5272/files#diff-1defdbd8a4c2ab55fb62ad44e9b314a8R256

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405241238
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
 
 Review comment:
   Maybe it is better to move this pattern to `contrib/dnnl.py`. Let me know.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612242010
 
 
   @masahi just a reminder, probably don't rebase again the master, other changes are here 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405843653
 
 

 ##########
 File path: src/relay/backend/utils.h
 ##########
 @@ -152,8 +152,35 @@ inline bool IsOp(const CallNode* call, const std::string& op_name) {
   Op op = GetRef<Op>(op_node);
   return op == Op::Get(op_name);
 }
+
+/*!
+ * \brief Retrieve the "root" conv2d op nested inside a fused call, such as conv2d + relu.
+ * \param call A Relay call node. Typically nn.relu when called the first time.
+ * \param depth The number of calls before conv2d call, counting from current_call.
+ * \param expected_op_names The names of ops in this fused call. Example: {"nn.conv2d", "add",
+ * "nn.relu"}
+ * \return conv2d op at the root
+ */
+
+inline const CallNode* GetRootConv2DCall(const CallNode* current_call, int depth,
+                                         const std::vector<std::string>& expected_op_names) {
+  CHECK(current_call && depth >= 0);
+
+  if (depth == 0) {
+    CHECK(IsOp(current_call, "nn.conv2d"));
+    return current_call;
+  }
+
+  CHECK(depth < expected_op_names.size() && IsOp(current_call, expected_op_names[depth]));
+  CHECK_GT(current_call->args.size(), 0);
+
+  const auto* next_call = current_call->args[0].as<CallNode>();
+  return GetRootConv2DCall(next_call, depth - 1, expected_op_names);
+}
 
 Review comment:
   I'm not incredibly familiar with majority of relay ops, but would it be safe to also modify this to "GetRootCall" in the case where the root composite call is something other than "nn.conv2d" ("nn.dense" for example) ?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406493347
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,128 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_dnnl_fuse():
+    def make_pattern(with_bias=True):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        if with_bias:
+            conv_out = relay.add(conv, bias)
 
 Review comment:
   Understood, thank you.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611758714
 
 
   > Although I still feel this is a bit ad-hoc, it should be fine for now as we only use DNNL as an example to demonstrate external codegen.
   
   What is still ad-hoc? I think I explained the motivation clearly in the original PR #4741 and the current impl using composite is the best we could do.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405782682
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   > @alexwong Could you give some more detail on the issue you're facing? I'd be happy to take a look.
   
   An example is I have a very simple Relay graph here where conv2d and clip are supported ops and squeeze and reshape are supported only if it's part of a squeeze->reshape pattern, so I am using mergecomposite for that.
   
   ```
   def @main(%data: Tensor[(1, 3, 224, 224), float32], %weight1: Tensor[(32, 3, 3, 3), float32]) -> Tensor[(3, 3, 3, 32), float32] {
     %0 = nn.conv2d(%data, %weight1, padding=[1, 1, 1, 1], kernel_size=[3, 3]) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %1 = clip(%0, a_min=0f, a_max=6f) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %2 = squeeze(%1) /* ty=Tensor[(32, 224, 224), float32] */;
     reshape(%2, newshape=[3, 3, 3, 32]) /* ty=Tensor[(3, 3, 3, 32), float32] */
   }
   ```
   
   After annotation I get:
   ```
   def @main(%data: Tensor[(1, 3, 224, 224), float32], %weight1: Tensor[(32, 3, 3, 3), float32]) -> Tensor[(3, 3, 3, 32), float32] {
     %0 = annotation.compiler_begin(%data, meta[relay.attrs.CompilerAttrs][0]) /* ty=Tensor[(1, 3, 224, 224), float32] */;
     %1 = annotation.compiler_begin(%weight1, meta[relay.attrs.CompilerAttrs][1]) /* ty=Tensor[(32, 3, 3, 3), float32] */;
     %2 = nn.conv2d(%0, %1, padding=[1, 1, 1, 1], kernel_size=[3, 3]) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %3 = annotation.compiler_end(%2, meta[relay.attrs.CompilerAttrs][2]) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %4 = annotation.compiler_begin(%3, meta[relay.attrs.CompilerAttrs][3]) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %5 = clip(%4, a_min=0f, a_max=6f) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %6 = annotation.compiler_end(%5, meta[relay.attrs.CompilerAttrs][4]) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %7 = annotation.compiler_begin(%6, meta[relay.attrs.CompilerAttrs][5]) /* ty=Tensor[(1, 32, 224, 224), float32] */;
     %9 = fn (%x: Tensor[(1, 32, 224, 224), float32], Composite="compilername.squeeze") -> Tensor[(3, 3, 3, 32), float32] {
       %8 = squeeze(%x) /* ty=Tensor[(32, 224, 224), float32] */;
       reshape(%8, newshape=[3, 3, 3, 32]) /* ty=Tensor[(3, 3, 3, 32), float32] */
     };
     %10 = %9(%7) /* ty=Tensor[(3, 3, 3, 32), float32] */;
     annotation.compiler_end(%10, meta[relay.attrs.CompilerAttrs][6]) /* ty=Tensor[(3, 3, 3, 32), float32] */
   }
   ```
   
   This looks right to me but mergecompilerregions is throwing the following error: 
   ```
   tvm._ffi.base.TVMError: TVMError: Cannot find the corresponding region for start annotation:
   v0.0.4
   free_var %x: Tensor[(1, 32, 224, 224), float32]
   %0 = annotation.compiler_begin(%x, meta[relay.attrs.CompilerAttrs][0]);
   %1 = squeeze(%0);
   %2 = annotation.compiler_end(%1, meta[relay.attrs.CompilerAttrs][1]);
   annotation.compiler_begin(%2, meta[relay.attrs.CompilerAttrs][2])
   // meta data omitted. you can use show_meta_data=True to include meta data
   ```
   
   Not sure if it's expected to not work with composite functions but I think there may be some missing logic there.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405768368
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   @alexwong I'll add another pattern in the test, conv2d + relu. I can try MergeCompilerRegions + AnnotateTarget passes in that example. This way we can demonstrate both approaches.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-610771579
 
 
   @tqchen I got segfault in CI that is not related to this PR. Can you have a look?
   https://ci.tvm.ai/blue/organizations/jenkins/tvm/detail/PR-5272/1/pipeline

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi merged pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi merged pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272
 
 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi edited a comment on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi edited a comment on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611758714
 
 
   > Although I still feel this is a bit ad-hoc, it should be fine for now as we only use DNNL as an example to demonstrate external codegen.
   
   What is still ad-hoc? I think I explained the motivation clearly in the original PR #4741 and the current impl using composite is the best we could do. Wanting fused op support in external codegen is a natural and @alexbooth already finds it useful.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405809268
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -30,14 +30,108 @@
 #include <tvm/runtime/registry.h>
 
 #include <fstream>
+#include <numeric>
 #include <sstream>
 
+#include "../../utils.h"
 #include "../codegen_c/codegen_c.h"
 
 namespace tvm {
 namespace relay {
 namespace contrib {
 
+using namespace backend;
+
+const CallNode* GetRootConv2DCall(const CallNode* relu_call) {
+  CHECK(relu_call && IsOp(relu_call, "nn.relu"));
+  const auto relu_arg = relu_call->args[0];
+  const CallNode* add_call = relu_arg.as<CallNode>();
+  CHECK(add_call && IsOp(add_call, "add"));
+  const auto add_arg = add_call->args[0];
+  const CallNode* conv_call = add_arg.as<CallNode>();
+  CHECK(conv_call && IsOp(conv_call, "nn.conv2d"));
+  return conv_call;
+}
 
 Review comment:
    Maybe this can be more generic, might get tedious to implement this kind of function if there are many different patterns. Could we just pass in a vector of op names like ["nn.relu", "add", "nn.conv2d"] along with the CallNode?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405843653
 
 

 ##########
 File path: src/relay/backend/utils.h
 ##########
 @@ -152,8 +152,35 @@ inline bool IsOp(const CallNode* call, const std::string& op_name) {
   Op op = GetRef<Op>(op_node);
   return op == Op::Get(op_name);
 }
+
+/*!
+ * \brief Retrieve the "root" conv2d op nested inside a fused call, such as conv2d + relu.
+ * \param call A Relay call node. Typically nn.relu when called the first time.
+ * \param depth The number of calls before conv2d call, counting from current_call.
+ * \param expected_op_names The names of ops in this fused call. Example: {"nn.conv2d", "add",
+ * "nn.relu"}
+ * \return conv2d op at the root
+ */
+
+inline const CallNode* GetRootConv2DCall(const CallNode* current_call, int depth,
+                                         const std::vector<std::string>& expected_op_names) {
+  CHECK(current_call && depth >= 0);
+
+  if (depth == 0) {
+    CHECK(IsOp(current_call, "nn.conv2d"));
+    return current_call;
+  }
+
+  CHECK(depth < expected_op_names.size() && IsOp(current_call, expected_op_names[depth]));
+  CHECK_GT(current_call->args.size(), 0);
+
+  const auto* next_call = current_call->args[0].as<CallNode>();
+  return GetRootConv2DCall(next_call, depth - 1, expected_op_names);
+}
 
 Review comment:
   I'm not incredibly familiar with majority of relay ops, but would it be safe to also modify this "GetRootCall" in the case where the root composite call is something other than "nn.conv2d" ("nn.dense" for example) ?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406382829
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   Any updates / initial findings?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405841053
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -30,14 +30,108 @@
 #include <tvm/runtime/registry.h>
 
 #include <fstream>
+#include <numeric>
 #include <sstream>
 
+#include "../../utils.h"
 #include "../codegen_c/codegen_c.h"
 
 namespace tvm {
 namespace relay {
 namespace contrib {
 
+using namespace backend;
+
+const CallNode* GetRootConv2DCall(const CallNode* relu_call) {
+  CHECK(relu_call && IsOp(relu_call, "nn.relu"));
+  const auto relu_arg = relu_call->args[0];
+  const CallNode* add_call = relu_arg.as<CallNode>();
+  CHECK(add_call && IsOp(add_call, "add"));
+  const auto add_arg = add_call->args[0];
+  const CallNode* conv_call = add_arg.as<CallNode>();
+  CHECK(conv_call && IsOp(conv_call, "nn.conv2d"));
+  return conv_call;
+}
 
 Review comment:
   Ok done. See the last commit.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405765549
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   I'm not familiar with what MergeCompilerRegions does, but assuming that it merges consecutive ops that are supported by the same compiler, yes it is possible in this case because all ops involved in the composite above, conv2d, add, and relu, are supported by the dnnl backend https://github.com/apache/incubator-tvm/blob/master/python/tvm/relay/op/contrib/dnnl.py#L59-L64. 
   
   But I think composite is supposed to be more general, it works even if `_register_external_op_helper("add")` is missing in dnnl.py above, for example. The purpose of composite is to target a "complex" op whose ops that constitute it are not necessary supported by the compiler by itself.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406962012
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   Yes, this PR fixed it!

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r406974254
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -53,163 +141,186 @@ class CodegenDNNL : public ExprVisitor, public CodegenCBase {
   }
 
   void VisitExpr_(const TupleGetItemNode* op) final {
-    // Do nothing
+    VisitExpr(op->tuple);
 
 Review comment:
   I wondered why this change is on my diff, but it seems TupleGetItemNode handling was reverted in #5277 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
comaniac commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612110937
 
 
   > What is still ad-hoc? I think I explained the motivation clearly in the original PR #4741 and the current impl using composite is the best we could do. Wanting fused op support in external codegen is a natural and @alexbooth already finds it useful.
   
   Sorry for being unclear. The motivation of using fused ops to demonstrate composite functions is definitely clear. What I meant was the general implementation of DNNL codegen as I've mentioned in #4741, so I'm good with this 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexbooth commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405858985
 
 

 ##########
 File path: src/relay/backend/utils.h
 ##########
 @@ -152,8 +152,35 @@ inline bool IsOp(const CallNode* call, const std::string& op_name) {
   Op op = GetRef<Op>(op_node);
   return op == Op::Get(op_name);
 }
+
+/*!
+ * \brief Retrieve the "root" conv2d op nested inside a fused call, such as conv2d + relu.
+ * \param call A Relay call node. Typically nn.relu when called the first time.
+ * \param depth The number of calls before conv2d call, counting from current_call.
+ * \param expected_op_names The names of ops in this fused call. Example: {"nn.conv2d", "add",
+ * "nn.relu"}
+ * \return conv2d op at the root
+ */
+
+inline const CallNode* GetRootConv2DCall(const CallNode* current_call, int depth,
+                                         const std::vector<std::string>& expected_op_names) {
+  CHECK(current_call && depth >= 0);
+
+  if (depth == 0) {
+    CHECK(IsOp(current_call, "nn.conv2d"));
+    return current_call;
+  }
+
+  CHECK(depth < expected_op_names.size() && IsOp(current_call, expected_op_names[depth]));
+  CHECK_GT(current_call->args.size(), 0);
+
+  const auto* next_call = current_call->args[0].as<CallNode>();
+  return GetRootConv2DCall(next_call, depth - 1, expected_op_names);
+}
 
 Review comment:
   Great! Yes this will work for majority of my use cases. If there is an easier way to find a composite function's root call, maybe someone else can comment. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405832751
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -30,14 +30,108 @@
 #include <tvm/runtime/registry.h>
 
 #include <fstream>
+#include <numeric>
 #include <sstream>
 
+#include "../../utils.h"
 #include "../codegen_c/codegen_c.h"
 
 namespace tvm {
 namespace relay {
 namespace contrib {
 
+using namespace backend;
+
+const CallNode* GetRootConv2DCall(const CallNode* relu_call) {
+  CHECK(relu_call && IsOp(relu_call, "nn.relu"));
+  const auto relu_arg = relu_call->args[0];
+  const CallNode* add_call = relu_arg.as<CallNode>();
+  CHECK(add_call && IsOp(add_call, "add"));
+  const auto add_arg = add_call->args[0];
+  const CallNode* conv_call = add_arg.as<CallNode>();
+  CHECK(conv_call && IsOp(conv_call, "nn.conv2d"));
+  return conv_call;
+}
 
 Review comment:
   fixed, please see the new change

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611211709
 
 
   Also added support for conv2d + relu pattern. This shows how easy to add a new pattern and support it in the codegen.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
comaniac commented on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-612261195
 
 
   > Changes to multi output support in dnnl/codegen.cc has also been reverted in #5277, but I've already integrated that change in this PR last night.
   
   Yes I've also checked that those diffs are in this PR as well. Thanks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405759315
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   I'm just curious but is it possible to try MergeCompilerRegions here? In my own testing, I seem to be having issues with composite functions and that pass. 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
alexwong commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405788230
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   That'd be great! Let me know your initial findings, might be something small :)

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi edited a comment on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi edited a comment on issue #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#issuecomment-611211709
 
 
   Also added support for conv2d + relu pattern. This shows how easy it is to add a new pattern and support it in the codegen.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405813179
 
 

 ##########
 File path: src/relay/backend/contrib/dnnl/codegen.cc
 ##########
 @@ -133,83 +209,100 @@ class CodegenDNNL : public ExprVisitor, public CodegenCBase {
   }
 
  private:
-  std::vector<std::string> Conv2d(const CallNode* call) {
-    std::vector<std::string> args;
-    const auto* conv2d_attr = call->attrs.as<Conv2DAttrs>();
-    CHECK(conv2d_attr);
-
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
+  struct GenerateBodyOutput {
+    std::string decl, buf;
+    int out_size = 1;
+    std::string out;
+  };
 
-    // Args: N, C, H, W
-    for (auto s : ishape) {
-      args.push_back(std::to_string(s));
+  std::vector<std::string> GetArgumentNames(const CallNode* call) {
+    std::vector<std::string> arg_names;
+    for (size_t i = 0; i < call->args.size(); ++i) {
+      VisitExpr(call->args[i]);
+      for (auto out : out_) {
+        arg_names.push_back(out.name);
+      }
     }
-
-    // Args: O, G, Ph, Pw, Kh, Kw, Sh, Sw
-    args.push_back(std::to_string(wshape[0]));
-    args.push_back(std::to_string(conv2d_attr->groups));
-    args.push_back(std::to_string(conv2d_attr->padding[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->padding[1].as<IntImmNode>()->value));
-    args.push_back(std::to_string(wshape[2]));
-    args.push_back(std::to_string(wshape[3]));
-    args.push_back(std::to_string(conv2d_attr->strides[0].as<IntImmNode>()->value));
-    args.push_back(std::to_string(conv2d_attr->strides[1].as<IntImmNode>()->value));
-
-    return args;
+    return arg_names;
   }
 
-  std::vector<std::string> Dense(const CallNode* call) {
-    std::vector<std::string> args;
-    auto ishape = GetShape(call->args[0]->checked_type());
-    auto wshape = GetShape(call->args[1]->checked_type());
-
-    // Args: N, C, O
-    args.push_back(std::to_string(ishape[0]));
-    args.push_back(std::to_string(ishape[1]));
-    args.push_back(std::to_string(wshape[0]));
+  GenerateBodyOutput GenerateOpCall(const CallNode* call) {
+    const auto* op_node = call->op.as<OpNode>();
+    CHECK(op_node) << "OpNode expected, got something else";
 
 Review comment:
   Let's also print out what the op_node is
   ```
   CHECK(op_node) << "Expect OpNode, but got " << op_node->GetTypeKey();
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405850469
 
 

 ##########
 File path: src/relay/backend/utils.h
 ##########
 @@ -152,8 +152,35 @@ inline bool IsOp(const CallNode* call, const std::string& op_name) {
   Op op = GetRef<Op>(op_node);
   return op == Op::Get(op_name);
 }
+
+/*!
+ * \brief Retrieve the "root" conv2d op nested inside a fused call, such as conv2d + relu.
+ * \param call A Relay call node. Typically nn.relu when called the first time.
+ * \param depth The number of calls before conv2d call, counting from current_call.
+ * \param expected_op_names The names of ops in this fused call. Example: {"nn.conv2d", "add",
+ * "nn.relu"}
+ * \return conv2d op at the root
+ */
+
+inline const CallNode* GetRootConv2DCall(const CallNode* current_call, int depth,
+                                         const std::vector<std::string>& expected_op_names) {
+  CHECK(current_call && depth >= 0);
+
+  if (depth == 0) {
+    CHECK(IsOp(current_call, "nn.conv2d"));
+    return current_call;
+  }
+
+  CHECK(depth < expected_op_names.size() && IsOp(current_call, expected_op_names[depth]));
+  CHECK_GT(current_call->args.size(), 0);
+
+  const auto* next_call = current_call->args[0].as<CallNode>();
+  return GetRootConv2DCall(next_call, depth - 1, expected_op_names);
+}
 
 Review comment:
   ah thanks that is a good idea since nothing there is specific to conv2d. We could also make `expected_op_names` arg optional, so that this can be used to retrieve any ancestor node at height `depth` above from `current_call`. If this sounds useful outside of fused call use cases, I can make it more general like that. What do you think? I can certainly make it work with dense or other `root_op`. cc @zhiics 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405769799
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   @mbaret If I choose MergeCompilerRegions + AnnotateTarget approach, can I still detect fused op inside codegen easily, like composite case?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op

Posted by GitBox <gi...@apache.org>.
mbaret commented on a change in pull request #5272: [BYOC] Add example of Composite + Annotate for DNNL fused op
URL: https://github.com/apache/incubator-tvm/pull/5272#discussion_r405774643
 
 

 ##########
 File path: tests/python/relay/test_pass_partition_graph.py
 ##########
 @@ -856,6 +857,111 @@ def expected():
     partitioned = transform.PartitionGraph()(mod)
     assert tvm.ir.structural_equal(partitioned, ref_mod, map_free_vars=True)
 
+
+def test_partition_conv_bias_relu():
+    def make_pattern():
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        weight = relay.var("weight")
+        bias = relay.var("bias")
+        conv = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                               channels=8, padding=(1, 1))
+        add = relay.add(conv, bias)
+        return relay.nn.relu(add)
+
+    def get_blocks(prefix, data, in_channel, out_channel,
+                   include_bn=True, include_sigmoid=False):
+        weight = relay.var(prefix + "weight")
+        bn_gamma = relay.var(prefix + "bn_gamma")
+        bn_beta = relay.var(prefix + "bn_beta")
+        bn_mmean = relay.var(prefix + "bn_mean")
+        bn_mvar = relay.var(prefix + "bn_var")
+
+        layer = relay.nn.conv2d(data=data, weight=weight, kernel_size=(3, 3),
+                                channels=out_channel, padding=(1, 1))
+        if include_bn:
+            bn_output = relay.nn.batch_norm(layer, bn_gamma, bn_beta,
+                                            bn_mmean, bn_mvar)
+            layer = bn_output[0]
+        if include_sigmoid:
+            # dummy layer to prevent pattern detection
+            layer = relay.sigmoid(layer)
+        layer = relay.nn.relu(layer)
+        return layer
+
+    def get_net(include_bn=True, include_sigmoid=False):
+        data = relay.var("data", relay.TensorType((1, 3, 224, 224), "float32"))
+        layer1 = get_blocks("layer1_", data, 3, 8, include_bn, include_sigmoid)
+        layer2 = get_blocks("layer2_", layer1, 8, 8, include_bn, include_sigmoid)
+        return relay.Function(relay.analysis.free_vars(layer2), layer2)
+
+    def get_partitoned_mod(mod, params):
+        # This is required for constant folding
+        mod["main"] = bind_params_by_name(mod["main"], params)
+        pattern_table = [
+            ("dnnl.conv_bias_relu", make_pattern())
+        ]
+        remove_bn_pass = transform.Sequential([
+            transform.InferType(),
+            transform.SimplifyInference(),
+            transform.FoldConstant(),
+            transform.FoldScaleAxis(),
+        ])
+        composite_partition = transform.Sequential([
+            remove_bn_pass,
+            transform.MergeComposite(pattern_table),
+            transform.AnnotateTarget("dnnl"),
 
 Review comment:
   The motivation behind MergeCompilerRegions is when you have a codegen that wants to do its own graph-level optimisations, so it should see the full offloaded subgraph rather than one op at a time.

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


With regards,
Apache Git Services