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/07/17 08:41:23 UTC

[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5915: [BYOC][Contrib] Arm Compute Library integration

mbaret commented on a change in pull request #5915:
URL: https://github.com/apache/incubator-tvm/pull/5915#discussion_r456115777



##########
File path: python/tvm/relay/op/contrib/arm_compute_lib.py
##########
@@ -0,0 +1,122 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# pylint: disable=invalid-name, unused-argument
+"""Arm Compute Library supported operators."""
+import tvm
+from tvm.relay import transform
+from tvm.relay.build_module import bind_params_by_name
+
+from ...dataflow_pattern import wildcard, is_op, is_constant
+from .register import register_pattern_table
+
+
+def is_arm_compute_runtime_enabled():
+    """Check if the ACL graph runtime is present.
+
+    Returns
+    -------
+    ret: bool
+        True if present, False if not.
+    """
+    check_enabled = tvm.get_global_func("relay.op.is_arm_compute_runtime_enabled", True)
+    if check_enabled:
+        return check_enabled()
+    return False
+
+
+def partition_for_arm_compute_lib(mod, params=None):
+    """Partition the graph greedily offloading supported
+    operators to Arm Compute Library.
+
+    Parameters
+    ----------
+    mod : Module
+        The module to run passes on.
+    params : Optional[Dict[str, NDArray]]
+        Constant input parameters.
+
+    Returns
+    -------
+    ret : annotated and partitioned module.
+    """
+    if params:
+        mod['main'] = bind_params_by_name(mod['main'], params)
+
+    seq = tvm.transform.Sequential([transform.MergeComposite(arm_compute_lib_pattern_table()),
+                                    transform.AnnotateTarget('arm_compute_lib'),
+                                    transform.PartitionGraph()])
+
+    return seq(mod)
+
+
+@register_pattern_table("arm_compute_lib")
+def arm_compute_lib_pattern_table():
+    """Get the ACL pattern table."""
+
+    def conv_pattern():
+        """Create a convolution pattern.
+
+        Returns
+        -------
+        pattern : dataflow_pattern.AltPattern
+            Denotes the convolution pattern.
+        """
+        pattern = is_op('nn.pad')(wildcard()) | wildcard()
+        pattern = is_op('nn.conv2d')(pattern, is_constant())
+        pattern = pattern.optional(lambda x: is_op('nn.bias_add')(x, is_constant()))
+        pattern = pattern.optional(is_op('nn.relu'))
+        return pattern
+
+    def check_conv(extract):
+        """Check conv pattern is supported by ACL."""
+        call = extract
+        while call.op.name != "nn.conv2d":
+            call = call.args[0]
+        return conv2d(call.attrs, call.args)
+
+    return [('arm_compute_lib.conv2d', conv_pattern(), check_conv)]
+
+
+def _register_external_op_helper(op_name, supported=True):
+    @tvm.ir.register_op_attr(op_name, "target.arm_compute_lib")
+    def _func_wrapper(attrs, args):
+        return supported
+
+    return _func_wrapper
+
+
+_register_external_op_helper("reshape")
+
+
+@tvm.ir.register_op_attr("nn.conv2d", "target.arm_compute_lib")
+def conv2d(attrs, args):
+    """Check if the external ACL codegen for conv2d should be used."""
+    if attrs.groups != 1:
+        return False
+    if attrs.data_layout != "NHWC":
+        return False

Review comment:
       Pedantically you may want to check for data type here as well (presumably = float32).

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/arm_compute_lib/codegen_acl.cc

Review comment:
       codegen_acl.cc -> codegen.cc

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/runtime/contrib/arm_compute_lib/acl_runtime.cc
+ * \brief A simple JSON runtime for Arm Compute Library.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif
+
+class ACLRuntime : public JSONRuntimeBase {
+ public:
+  /*!
+   * \brief The ACL runtime module. Deserialize the provided functions
+   * on creation and store in the layer cache.
+   *
+   * \param symbol_name The name of the function.
+   * \param graph_json serialized JSON representation of a sub-graph.
+   * \param const_names The names of each constant in the sub-graph.
+   */
+  explicit ACLRuntime(const std::string& symbol_name, const std::string& graph_json,
+                      const Array<String>& const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  /*!
+   * \brief The type key of the module.
+   *
+   * \return module type key.
+   */
+  const char* type_key() const override { return "arm_compute_lib"; }
+
+  /*!
+   * \brief Initialize runtime. Create ACL layer from JSON
+   * representation.
+   *
+   * \param consts The constant params from compiled model.
+   */
+  void Init(const Array<NDArray>& consts) override {
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+    SetupConstants(consts);
+    BuildEngine();
+  }
+
+  /*!
+   * \brief Get the JSON generated by codegen.
+   *
+   * \param format the format to return (only JSON for the time being)
+   * \return A string of JSON.
+   */
+  std::string GetSource(const std::string& format) override {
+    if (format == "json") {
+      return graph_json_;
+    }
+    LOG(FATAL) << "Format not supported by Arm Compute Library runtime.";
+    return "";
+  }
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+  /*!
+   * \brief Unpack inputs and outputs and run inference on a given layer.
+   *
+   * \param args Access inputs and outputs.
+   * \param function The layer to execute inference on.
+   * \return Status of inference.
+   */
+  void Run() override {
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto nid = input_nodes_[i];
+      uint32_t eid = EntryID(nid, 0);
+      if (nodes_[nid].GetOpType() == "input") {
+        void* data = data_entry_[eid]->data;
+        CheckACLError(layer_.inputs[i].allocator()->import_memory(data));
+      }
+    }
+
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      uint32_t eid = EntryID(outputs_[i]);
+      void* data = data_entry_[eid]->data;
+      CheckACLError(layer_.outputs[i].allocator()->import_memory(data));
+    }
+
+    this->layer_.function->run();
+  }
+
+ private:
+  /*!
+   * \brief Build ACL layer from JSON representation and cache.
+   *
+   * \note For the time being only one layer or operator is supported
+   * per engine.
+   */
+  void BuildEngine() {
+    std::shared_ptr<arm_compute::MemoryManagerOnDemand> mm = MakeMemoryManager();
+    int num_pools = 0;
+
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      uint32_t nid = input_nodes_[i];
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "input") {
+        layer_.inputs.push_back(MakeTensor(node));
+      } else if (node.GetOpType() == "const") {
+        uint32_t eid = EntryID(nid, 0);
+        void* data = data_entry_[eid]->data;
+        layer_.const_inputs.push_back(MakeTensor(node, data));
+      }
+    }
+
+    bool found_kernel_node = false;
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (found_kernel_node) {
+        LOG(FATAL)
+            << "Arm Compute Library runtime module only supports one kernel node per function.";
+      }
+      if (node.GetOpType() == "kernel") {
+        found_kernel_node = true;
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name || "arm_compute_lib.conv2d" == op_name) {

Review comment:
       Why either or here? Could all conv2ds not be arm_compute_lib.conv2ds?

##########
File path: src/relay/backend/contrib/arm_compute_lib/codegen.cc
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/relay/backend/contrib/arm_compute_lib/codegen_acl.cc
+ * \brief Implementation of the Relay -> ACL JSON serializer.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/attrs/nn.h>
+#include <tvm/relay/type.h>
+
+#include "../../utils.h"
+#include "codegen_acl.h"
+
+namespace tvm {
+namespace relay {
+namespace contrib {
+namespace arm_compute_lib {
+
+using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+std::vector<JSONGraphNodeEntry> ACLJSONSerializer::VisitExpr_(const CallNode* cn) {
+  Expr expr = GetRef<Expr>(cn);
+  std::string name;
+  std::shared_ptr<JSONGraphNode> json_node;
+
+  if (cn->op.as<OpNode>()) {
+    json_node = CreateOpJSONNode(cn);
+  } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+    auto comp = fn->GetAttr<String>(attr::kComposite);
+    CHECK(comp.defined()) << "Arm Compute Library JSON runtime only supports composite functions.";
+    name = comp.value();
+    if (name == "arm_compute_lib.conv2d") {
+      json_node = CreateCompositeConvJSONNode(cn);
+    } else {
+      LOG(FATAL) << "Unrecognized Arm Compute Library pattern: " << name;
+    }
+  } else {
+    LOG(FATAL) << "Arm Compute Library JSON runtime does not support calls to "
+               << cn->op->GetTypeKey();
+  }
+
+  return AddNode(json_node, GetRef<Expr>(cn));
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateOpJSONNode(const CallNode* cn) {
+  const auto* op = cn->op.as<OpNode>();
+  CHECK(op);
+  const std::string name = op->name;
+  // Collect inputs
+  std::vector<JSONGraphNodeEntry> inputs;
+  for (const auto& arg : cn->args) {
+    auto res = VisitExpr(arg);
+    inputs.insert(inputs.end(), res.begin(), res.end());
+  }
+  // Create JSON op
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, cn);
+  return json_node;
+}
+
+std::shared_ptr<JSONGraphNode> ACLJSONSerializer::CreateCompositeConvJSONNode(const CallNode* cn) {
+  const std::string name = "arm_compute_lib.conv2d";
+  const CallNode* pad = nullptr;
+  const CallNode* conv;
+  const CallNode* bias = nullptr;
+  bool has_activation = false;
+
+  // Unpack composite function
+  const auto* fn = cn->op.as<FunctionNode>();
+  CHECK(fn);
+  const auto* current_call = fn->body.as<CallNode>();
+  if (backend::IsOp(current_call, "nn.relu")) {
+    has_activation = true;
+    current_call = current_call->args[0].as<CallNode>();
+  }
+  if (backend::IsOp(current_call, "nn.bias_add")) {
+    bias = current_call;
+    current_call = current_call->args[0].as<CallNode>();
+  }
+  CHECK(backend::IsOp(current_call, "nn.conv2d"));
+  conv = current_call;
+  if (!current_call->args.empty() && current_call->args[0]->IsInstance<CallNode>()) {
+    current_call = current_call->args[0].as<CallNode>();
+    if (backend::IsOp(current_call, "nn.pad")) {
+      pad = current_call;
+    }
+  }
+
+  const auto* conv_attr = conv->attrs.as<Conv2DAttrs>();
+  CHECK(conv_attr);
+  CHECK(conv_attr->kernel_layout == "OHWI")
+      << "Kernel layout must be OHWI, has the module been pre-processed correctly?";
+
+  std::vector<JSONGraphNodeEntry> inputs;
+  inputs.push_back(VisitExpr(cn->args[0])[0]);
+  inputs.push_back(VisitExpr(conv->args[1])[0]);
+  if (bias) {
+    inputs.push_back(VisitExpr(bias->args[1])[0]);
+  }
+
+  auto json_node = std::make_shared<JSONGraphNode>(name, "kernel", inputs, 1);
+  SetCallNodeAttribute(json_node, conv);
+
+  // Override attributes
+  if (pad) {
+    const auto* pad_attr = pad->attrs.as<PadAttrs>();
+    CHECK(pad_attr);
+    auto p = pad_attr->pad_width;
+    // Convert to TVM layout for now, conversion to ACL layout takes place in runtime.
+    // Standard convolution pad layout for TVM: top, left, bottom, right.
+    std::vector<std::string> padding = {std::to_string(p[1][0].as<IntImmNode>()->value),
+                                        std::to_string(p[2][0].as<IntImmNode>()->value),
+                                        std::to_string(p[1][1].as<IntImmNode>()->value),
+                                        std::to_string(p[2][1].as<IntImmNode>()->value)};
+    std::vector<dmlc::any> padding_attr;
+    padding_attr.emplace_back(padding);
+    json_node->SetAttr("padding", padding_attr);
+  }
+  if (has_activation) {
+    std::vector<std::string> activation_type = {"relu"};
+    std::vector<dmlc::any> act_attr;
+    act_attr.emplace_back(activation_type);
+    json_node->SetAttr("activation_type", act_attr);
+  }
+  return json_node;
+}
+
+IRModule PreProcessModule(const IRModule& mod) {
+  IRModule preprocessed_module;
+  tvm::Map<String, Array<String>> desired_layouts = {{"nn.conv2d", {"NHWC", "OHWI"}}};
+  preprocessed_module = transform::ConvertLayout(desired_layouts)(mod);
+  preprocessed_module = transform::FoldConstant()(preprocessed_module);
+  return preprocessed_module;
+}

Review comment:
       Does this mean if ACL offload happens the entire graph (including bits not destined for ACL) get converted to NHWC, or is it just ACL partitions?

##########
File path: src/runtime/contrib/arm_compute_lib/acl_runtime.cc
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file src/runtime/contrib/arm_compute_lib/acl_runtime.cc
+ * \brief A simple JSON runtime for Arm Compute Library.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include "../../file_util.h"
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+#include <arm_compute/core/Types.h>
+#include <arm_compute/runtime/NEON/functions/NEConvolutionLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEPoolingLayer.h>
+#include <arm_compute/runtime/NEON/functions/NEReshapeLayer.h>
+
+#include "acl_allocator.h"
+#include "acl_utils.h"
+#endif
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+#ifdef TVM_GRAPH_RUNTIME_ARM_COMPUTE_LIB
+using namespace arm_compute_lib;
+#endif

Review comment:
       Not sure I understand this.




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