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/06/24 18:02:11 UTC

[GitHub] [incubator-tvm] comaniac opened a new pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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


   RFC discussion: https://discuss.tvm.ai/t/byoc-runtime-json-runtime-for-byoc/6579
   
   Currently, BYOC allows developers to choose either C source module or their customized module as the runtime for their accelerators. While we have provided an end-to-end execution flow of DNNL (i.e., MKL-DNN, OneDNN) using C source module, we found that many developers prefer to use a customized module to better integrate to their own runtime engine, such as TensorRT. As a result, this PR (collaborating with @zhiics) provides an end-to-end flow of DNNL using JSON runtime. Some detail highlights:
   
   * We provide JSON codegen and JSON runtime base classes. JSON codegen serializes a Relay subgaph to a JSON file; while JSON runtime base provides deserialization methods to interprete subgraphs in JSON format. Developers can derive JSON codegen to easily customize their codegen, or even directly use JSON codegen if their runtime engine accepts standard TVM graph runtime JSON.
   
   * We make a case study of leveraging JSON runtime with DNNL. The DNNL JSON runtime now supports conv2d, dense, relu, batch_norm, and add. As a result, it is able to run MobileNet. Note that DNNL JSON runtime only creates one DNNL execution engine for a subgraph, so it is much more efficient compared to the C source module version, which creates a DNNL engine for each *operator* in a subgraph.
   
   * DNNL JSON runtime handles constant tensors following the new mechanism in #5770.
   
   * DNNL codegen with C source module will be preserved for illustraction purpose, and we use cmake to control which DNNL codegen should be used. Specifically, `USE_DNNL_CODEGEN ON` and `USE_DNNL_CODEGEN JSON` enable the JSON runtime (and this is the default runtime for DNNL). When following the tutorial, which we will update after this PR, users may use `USE_DNNL_CODEGEN C_SRC` to enable C source module so that they can learn how it work.
   
   cc @masahi @mbaret @tqchen 


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: tests/python/relay/test_json_runtime.py
##########
@@ -0,0 +1,625 @@
+# 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.
+"""Unit tests for JSON codegen and runtime."""
+import os
+import sys
+
+import numpy as np
+
+import tvm
+import tvm.relay.op as reg
+import tvm.relay.testing
+from tvm import relay, runtime
+from tvm.contrib import util
+from tvm.relay import transform
+from tvm.relay.backend import compile_engine
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.op.contrib.register import get_pattern_table
+
+
+def set_func_attr(func, compile_name, symbol_name):
+    func = func.with_attr("Primitive", tvm.tir.IntImm("int32", 1))
+    func = func.with_attr("Inline", tvm.tir.IntImm("int32", 1))
+    func = func.with_attr("Compiler", compile_name)
+    func = func.with_attr("global_symbol", symbol_name)
+    return func
+
+
+def check_result(mod,
+                 ref_mod,
+                 map_inputs,
+                 out_shape,
+                 tol=1e-5,
+                 target="llvm",
+                 ctx=tvm.cpu(),
+                 params=None):
+    if sys.platform == "win32":
+        print("Skip test on Windows for now")
+        return
+
+    # Run the reference result
+    compile_engine.get().clear()
+    with relay.build_config(opt_level=3):

Review comment:
       Good catch. Wil 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



[GitHub] [incubator-tvm] lhutton1 commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: tests/python/relay/test_json_runtime.py
##########
@@ -0,0 +1,625 @@
+# 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.
+"""Unit tests for JSON codegen and runtime."""
+import os
+import sys
+
+import numpy as np
+
+import tvm
+import tvm.relay.op as reg
+import tvm.relay.testing
+from tvm import relay, runtime
+from tvm.contrib import util
+from tvm.relay import transform
+from tvm.relay.backend import compile_engine
+from tvm.relay.build_module import bind_params_by_name
+from tvm.relay.op.contrib.register import get_pattern_table
+
+
+def set_func_attr(func, compile_name, symbol_name):
+    func = func.with_attr("Primitive", tvm.tir.IntImm("int32", 1))
+    func = func.with_attr("Inline", tvm.tir.IntImm("int32", 1))
+    func = func.with_attr("Compiler", compile_name)
+    func = func.with_attr("global_symbol", symbol_name)
+    return func
+
+
+def check_result(mod,
+                 ref_mod,
+                 map_inputs,
+                 out_shape,
+                 tol=1e-5,
+                 target="llvm",
+                 ctx=tvm.cpu(),
+                 params=None):
+    if sys.platform == "win32":
+        print("Skip test on Windows for now")
+        return
+
+    # Run the reference result
+    compile_engine.get().clear()
+    with relay.build_config(opt_level=3):

Review comment:
       Better to replace with tvm.transform.PassContext?

##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       I feel like I might be missing something but in order to get the dnnl tests to run I had to add `DMLC_JSON_ENABLE_ANY(std::vector<std::string>, ...); ...` etc. Is this expected?  




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/graph_runtime_codegen.cc
##########
@@ -661,6 +660,8 @@ struct Handler<std::unordered_map<std::string, dmlc::any>> {
         writer->WriteObjectKeyValue(k, dmlc::get<std::vector<std::vector<int64_t>>>(v));
       } else if (SameType<std::vector<std::string>>(v)) {
         writer->WriteObjectKeyValue(k, dmlc::get<std::vector<std::string>>(v));
+      } else if (SameType<std::vector<dmlc::any>>(v)) {

Review comment:
       Thanks for catching on this. This change supports any type of vectors to be serialized and we should keep it in `json_node.h`. I'll push a commit to fix 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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/dnnl/codegen.cc
##########
@@ -417,13 +424,87 @@ class DNNLModuleCodegen : public CSourceModuleCodegenBase {
   std::ostringstream code_stream_;
 };
 
+#else  // DNNL JSON runtime
+
+class DNNLJSONSerializer : public backend::contrib::JSONSerializer {
+  using JSONGraphNode = tvm::runtime::json::JSONGraphNode;
+  using JSONGraphNodeEntry = tvm::runtime::json::JSONGraphNodeEntry;
+
+ public:
+  DNNLJSONSerializer(const std::string& symbol, const Expr& expr) : JSONSerializer(symbol, expr) {}
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) override {
+    Expr expr = GetRef<Expr>(cn);
+    std::string name;
+    const CallNode* call = cn;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto comp = fn->GetAttr<String>(attr::kComposite);
+      CHECK(comp.defined()) << "DNNL JSON runtime only supports composite functions.";
+      name = comp.value().operator std::string();

Review comment:
       remove `operator std::string()`




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       Hmm...we hit this before and that's why we put the above changes. Let me double check




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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-652642564


   @masahi Thanks, my comment should be resolved with a followup 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



[GitHub] [incubator-tvm] masahi commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
masahi commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-649146715


   maybe we should enable dnnl on CI?


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

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



[GitHub] [incubator-tvm] lhutton1 edited a comment on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
lhutton1 edited a comment on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-651268600


   Possibly out of scope for this PR but is there a plan to support multiple functions/sub-graphs? Currently it looks like there is only support for a single dnnl sub-graph after the graph is partitioned?


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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();
+};
+
+/*! \brief Serialize a Relay expression to JSON. */
+class JSONSerializer : public MemoizedExprTranslator<std::vector<JSONGraphNodeEntry>> {
+ public:
+  /*!
+   * \brief Constructor
+   *
+   * \param symbol The symbol that represents the graph being converted.
+   * \param expr The Relay expression to be converted to the JSON form.
+   */
+  JSONSerializer(const std::string& symbol, const Expr& expr) : symbol_(symbol), func_(expr) {}
+
+  void serialize() {
+    relay::Function func = Downcast<relay::Function>(func_);
+    // First we convert all the parameters into input nodes.
+    for (const auto& param : func->params) {
+      auto node_ptr = std::make_shared<JSONGraphNode>(param->name_hint(), "input" /* op_type_ */);
+      memo_[param] = AddNode(node_ptr, param);
+    }
+    heads_ = VisitExpr(func->body);
+  }
+
+  /*!\brief Return the required params. */
+  Array<String> GetParams() const { return params_; }
+
+  /*!\brief Return the generated json. */
+  std::string GetJSON() {
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    Save(&writer);
+    return os.str();
+  }
+
+ protected:
+  /*!
+   * \brief Add a node to graph.
+   *
+   * \param node A graph node. It is a shared pointer. Some attributes of it
+   *        will be added, i.e. shape and type. These attributes are attached to
+   *        the JSON graph in the end.
+   * \param expr The relay expression.
+   * \return A list of graph entry nodes. It the relay expr is a tuple type, we
+   *         will flatten it.
+   */
+  std::vector<JSONGraphNodeEntry> AddNode(JSONGraphObjectPtr node, const Expr& expr) {
+    auto checked_type = expr->checked_type();
+    auto node_id = nodes_.size();
+    nodes_.push_back(node);
+    std::vector<JSONGraphNodeEntry> ret;
+    ShapeVector shape;
+    TypeVector dtype;
+    // Flatten tuple node.
+    if (const auto* tuple_type = checked_type.as<TupleTypeNode>()) {
+      for (size_t i = 0; i < tuple_type->fields.size(); ++i) {
+        const auto* tensor_type = tuple_type->fields[i].as<TensorTypeNode>();
+        CHECK(tensor_type) << "Expect TensorType, but received: ."
+                           << tuple_type->fields[i]->GetTypeKey();
+        ret.push_back(JSONGraphNodeEntry(node_id, i));
+        shape.emplace_back(GetIntShape(tensor_type->shape));
+        dtype.emplace_back(DType2String(tensor_type->dtype));
+      }
+      node->SetNumOutput(tuple_type->fields.size());
+    } else {
+      const auto* tensor_type = checked_type.as<TensorTypeNode>();
+      CHECK(tensor_type) << "Expect TensorType, but received: " << checked_type->GetTypeKey();
+      shape.emplace_back(GetIntShape(tensor_type->shape));
+      dtype.emplace_back(DType2String(tensor_type->dtype));
+      ret.push_back(JSONGraphNodeEntry(node_id, 0));
+    }
+    std::vector<dmlc::any> shape_attrs;
+    shape_attrs.emplace_back(shape);
+    node->SetAttr("shape", shape_attrs);
+
+    std::vector<dmlc::any> type_attrs;
+    type_attrs.emplace_back(dtype);
+    node->SetAttr("dtype", type_attrs);
+    return ret;
+  }
+
+  void SetCallNodeAttribute(JSONGraphObjectPtr node, const CallNode* cn) {
+    if (cn->op.as<OpNode>()) {
+      OpAttrExtractor extractor(node);
+      const Object* call_attr = cn->attrs.get();
+      extractor.Extract(const_cast<Object*>(call_attr));
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto pattern = fn->GetAttr<String>(attr::kPartitionedFromPattern);
+      CHECK(pattern.defined());
+      std::vector<std::string> values;
+      values.push_back(pattern.value().operator std::string());
+      std::vector<dmlc::any> attr;
+      attr.emplace_back(values);
+      node->SetAttr("PartitionedFromPattern", attr);
+    }
+  }
+
+  std::vector<JSONGraphNodeEntry> VisitExprDefault_(const Object* op) {
+    LOG(FATAL) << "JSON runtime currently doesn't support " << op->GetTypeKey();
+    return {};
+  }
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const VarNode* vn) {
+    CHECK(memo_.count(GetRef<Expr>(vn)));
+    return memo_[GetRef<Expr>(vn)];
+  }
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const ConstantNode* cn) {
+    std::string name = symbol_ + "_const_" + std::to_string(params_.size());
+    params_.push_back(name);
+    auto node = std::make_shared<JSONGraphNode>(name, "const" /* op_type_ */);
+    return AddNode(node, GetRef<Expr>(cn));
+  }
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const TupleNode* tn) {
+    std::vector<JSONGraphNodeEntry> fields;
+    for (const auto& field : tn->fields) {
+      auto ref = VisitExpr(field);
+      fields.insert(fields.end(), ref.begin(), ref.end());
+    }
+    return fields;
+  }
+
+  std::vector<JSONGraphNodeEntry> VisitExpr_(const CallNode* cn) {
+    Expr expr = GetRef<Expr>(cn);
+    std::string name;
+    if (const auto* op_node = cn->op.as<OpNode>()) {
+      name = op_node->name;
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto comp = fn->GetAttr<String>(attr::kComposite);
+      CHECK(comp.defined()) << "JSON runtime only supports composite functions.";
+      name = comp.value().operator std::string();

Review comment:
       remove `operator std::string()`




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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});

Review comment:
       Yeah, this was to simplify serialization/deserialization




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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       It is not about current handler, it is about the specialization of struct Handler<std::vector<dmlc::any>>




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/metadata_module.cc
##########
@@ -48,15 +48,22 @@ class MetadataModuleNode : public ModuleNode {
  public:
   MetadataModuleNode(const std::unordered_map<std::string, NDArray>& metadata,
                      const std::unordered_map<std::string, std::vector<std::string>>& sym_vars)
-      : metadata_(metadata), sym_vars_(sym_vars) {}
+      : metadata_(metadata), sym_vars_(sym_vars) {
+    // Only the related submodules are cached to reduce the number of runtime
+    // symbol lookup for initialization. Otherwise, symbols/primitives in the
+    // DSO module will also be cached but they never need to be initialized.
+    for (const auto& it : sym_vars_) {
+      initialized_[it.first] = false;
+    }
+  }

Review comment:
       #5926 filed. Will removed these changes after it has been merged.




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
comaniac commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-651270339


   > Possibly out of scope for this PR but is there a plan to support multiple functions/sub-graphs? Currently it looks like there is only support for a single dnnl sub-graph after the graph is partitioned?
   
   We now have only one subgraph per module, but we could have many modules to support multiple subgraphs. Please see @mbaret 's comments to this PR and the discussions for details.


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/dnnl/dnnl_json_runtime.cc
##########
@@ -0,0 +1,456 @@
+/*
+ * 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/dnnl/dnnl_json_runtime.cc
+ * \brief A simple JSON runtime for DNNL.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+#include "dnnl.hpp"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime;
+using namespace tvm::runtime::json;
+
+class DNNLJSONRuntime : public JSONRuntimeBase {
+  using tag = dnnl::memory::format_tag;
+  using dt = dnnl::memory::data_type;
+
+ public:
+  DNNLJSONRuntime(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  const char* type_key() const { return "dnnl_json"; }
+
+  void Init(const Array<NDArray>& consts) override {
+    BuildEngine();
+
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+
+    // Setup constants entries for weights.
+    SetupConstants(consts);
+  }
+
+  void Run() override {
+    // Fill in the input buffers.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto eid = EntryID(input_nodes_[i], 0);
+      // TODO(@comaniac): Support other data lengths.
+      size_t offset_in_bytes = entry_out_mem_[eid].second * 4;
+      size_t buffer_size = GetDataSize(*data_entry_[eid]);
+      write_to_dnnl_memory(data_entry_[eid]->data, entry_out_mem_[eid].first, buffer_size,
+                           offset_in_bytes);
+    }
+
+    // Invoke the engine through intepreting the stream.
+    for (size_t i = 0; i < net_.size(); ++i) {
+      net_.at(i).execute(stream_, net_args_.at(i));
+    }
+    stream_.wait();
+
+    // Read output buffers.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      auto eid = EntryID(outputs_[i]);
+      size_t offset_in_bytes = entry_out_mem_[eid].second * 4;
+      size_t buffer_size = GetDataSize(*data_entry_[eid]);
+      read_from_dnnl_memory(data_entry_[eid]->data, entry_out_mem_[eid].first, buffer_size,
+                            offset_in_bytes);
+    }
+  }
+
+ private:
+  // Build up the engine based on the input graph.
+  void BuildEngine() {
+    engine_ = dnnl::engine(dnnl::engine::kind::cpu, 0);
+    stream_ = dnnl::stream(engine_);
+
+    // Build subgraph engine.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "kernel") {
+        CHECK_EQ(node.GetOpType(), "kernel");
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name) {
+          Conv2d(nid);
+        } else if ("dnnl.conv2d_relu" == op_name) {
+          Conv2d(nid, true, false);
+        } else if ("dnnl.conv2d_bias_relu" == op_name) {
+          Conv2d(nid, true, true);
+        } else if ("nn.dense" == op_name) {
+          Dense(nid);
+        } else if ("nn.batch_norm" == op_name) {
+          BatchNorm(nid);
+        } else if ("nn.relu" == op_name) {
+          Relu(nid);
+        } else if ("add" == op_name) {
+          Add(nid);
+        } else {
+          LOG(FATAL) << "Unsupported op: " << op_name;
+        }
+      }
+    }
+  }
+
+  // Bind a JSON graph node entry to a DNNL memory.
+  dnnl::memory BindDNNLMemory(const JSONGraphNodeEntry& entry, dnnl::memory::desc mem_desc,
+                              size_t offset = 0) {
+    auto eid = EntryID(entry);
+    if (entry_out_mem_.count(eid) == 0) {
+      return BindDNNLMemory(entry, dnnl::memory(mem_desc, engine_), offset);
+    }
+    return entry_out_mem_[eid].first;
+  }
+
+  // Bind a JSON graph node entry to a given DNNL memory.
+  dnnl::memory BindDNNLMemory(const JSONGraphNodeEntry& entry, dnnl::memory mem,
+                              size_t offset = 0) {
+    auto eid = EntryID(entry);
+    // Since the DNNL memory has been created before calling this function, we assume the entry
+    // has not yet been bind to the other DNNL memory; otherwise it may have memory leak.
+    CHECK_EQ(entry_out_mem_.count(eid), 0);
+
+    // TODO(@comanic): Support other data types (i.e., int8).
+    auto data_node = nodes_[entry.id_];
+    auto dltype = data_node.GetOpDataType()[entry.index_];
+    CHECK_EQ(dltype.bits, 32);
+
+    entry_out_mem_[eid] = {mem, offset};
+    return entry_out_mem_[eid].first;
+  }
+
+  void Conv2d(const size_t& nid, const bool has_relu = false, const bool has_bias = false) {
+    auto node = nodes_[nid];
+
+    // Setup attributes.
+    auto data_entry = node.GetInputs()[0];
+    auto weight_entry = node.GetInputs()[1];
+    dnnl::memory::dims input_shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    dnnl::memory::dims weight_shape = nodes_[weight_entry.id_].GetOpShape()[weight_entry.index_];
+    std::vector<std::string> str_strides = node.GetAttr<std::vector<std::string>>("strides");
+    std::vector<std::string> str_padding = node.GetAttr<std::vector<std::string>>("padding");
+    dnnl::memory::dim groups = std::stoi(node.GetAttr<std::vector<std::string>>("groups")[0]);
+
+    dnnl::memory::dim N = input_shape[0],       // batch size
+        IC = input_shape[1],                    // input channels
+        IH = input_shape[2],                    // input height
+        IW = input_shape[2],                    // input width
+        OC = weight_shape[0],                   // output channels
+        KH = weight_shape[2],                   // weight height
+        KW = weight_shape[3],                   // weight width
+        PH_L = std::stoi(str_padding[1]),       // height padding: left
+        PH_R = std::stoi(str_padding[3]),       // height padding: right
+        PW_L = std::stoi(str_padding[0]),       // width padding: left
+        PW_R = std::stoi(str_padding[2]),       // width padding: right
+        SH = std::stoi(str_strides[0]),         // height-wise stride
+        SW = std::stoi(str_strides[0]),         // weight-wise stride
+        OH = (IH - KH + PH_L + PH_R) / SH + 1,  // output height
+        OW = (IW - KW + PW_L + PW_R) / SW + 1;  // output width
+
+    // Memory shapes.
+    dnnl::memory::dims src_dims = {N, IC, IH, IW};
+    dnnl::memory::dims weights_dims = {OC, IC, KH, KW};
+    if (groups > 1) {
+      weights_dims = {groups, 1, IC / groups, KH, KW};
+    }
+    dnnl::memory::dims bias_dims = {OC};
+    dnnl::memory::dims dst_dims = {N, OC, OH, OW};
+    dnnl::memory::dims strides_dims = {SH, SW};
+    dnnl::memory::dims padding_dims_l = {PH_L, PW_L};
+    dnnl::memory::dims padding_dims_r = {PH_R, PW_R};
+
+    // Memory descriptions.
+    auto conv_src_md = dnnl::memory::desc(src_dims, dt::f32, tag::any);
+    auto conv_weights_md = dnnl::memory::desc(weights_dims, dt::f32, tag::any);
+    auto conv_bias_md = dnnl::memory::desc(bias_dims, dt::f32, tag::any);
+    auto conv_dst_md = dnnl::memory::desc(dst_dims, dt::f32, tag::nchw);
+
+    // Covn2d description.
+    auto conv_desc = dnnl::convolution_forward::desc(
+        dnnl::prop_kind::forward_inference, dnnl::algorithm::convolution_direct, conv_src_md,
+        conv_weights_md, conv_bias_md, conv_dst_md, strides_dims, padding_dims_l, padding_dims_r);
+
+    // Enable ReLU
+    dnnl::primitive_attr attr;
+    if (has_relu) {
+      dnnl::post_ops ops;
+      ops.append_eltwise(1.f, dnnl::algorithm::eltwise_relu, 0.f, 0.f);
+      attr.set_post_ops(ops);
+    }
+
+    auto conv2d_prim_desc = dnnl::convolution_forward::primitive_desc(conv_desc, attr, engine_);
+
+    // Push to the network.
+    auto conv = dnnl::convolution_forward(conv2d_prim_desc);
+    net_.push_back(conv);
+
+    // Data memory.
+    CHECK_EQ(node.GetAttr<std::vector<std::string>>("data_layout")[0], "NCHW");
+    auto conv2d_src_memory = BindDNNLMemory(data_entry, {src_dims, dt::f32, tag::nchw});
+
+    // Weight memory.
+    CHECK_EQ(node.GetAttr<std::vector<std::string>>("kernel_layout")[0], "OIHW");
+    auto conv2d_weights_memory = BindDNNLMemory(
+        weight_entry, {weights_dims, dt::f32, (groups > 1) ? tag::goihw : tag::oihw});
+
+    // Bias memory.
+    auto conv2d_bias_memory = dnnl::memory({bias_dims, dt::f32, tag::x}, engine_);
+    if (has_bias) {
+      auto bias_entry = node.GetInputs()[2];
+      BindDNNLMemory(bias_entry, conv2d_bias_memory);
+    } else {
+      float bias[OC] = {0};
+      write_to_dnnl_memory(bias, conv2d_bias_memory, OC * sizeof(float));
+    }
+
+    // Output memory.
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto conv2d_dst_memory = BindDNNLMemory(out_entry, conv2d_prim_desc.dst_desc());
+
+    // Bind memory buffers.
+    net_args_.push_back({{DNNL_ARG_SRC, conv2d_src_memory},
+                         {DNNL_ARG_WEIGHTS, conv2d_weights_memory},
+                         {DNNL_ARG_BIAS, conv2d_bias_memory},
+                         {DNNL_ARG_DST, conv2d_dst_memory}});
+  }
+
+  void Dense(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    // Setup attributes.
+    auto data_entry = node.GetInputs()[0];
+    auto weight_entry = node.GetInputs()[1];
+    dnnl::memory::dims input_shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    dnnl::memory::dims weight_shape = nodes_[weight_entry.id_].GetOpShape()[weight_entry.index_];
+
+    dnnl::memory::dim B = input_shape[0],  // batch size
+        IC = input_shape[1],               // input channels
+        OC = weight_shape[0];              // output channels
+
+    // Memory shapes.
+    dnnl::memory::dims data_dims = {B, IC};
+    dnnl::memory::dims weight_dims = {OC, IC};
+    dnnl::memory::dims bias_dims = {OC};
+    dnnl::memory::dims out_dims = {B, OC};
+
+    // Memory descriptions.
+    auto data_md = dnnl::memory::desc({data_dims, dt::f32, tag::nc});
+    auto weight_md = dnnl::memory::desc({weight_dims, dt::f32, tag::nc});
+    auto bias_md = dnnl::memory::desc({bias_dims, dt::f32, tag::x});
+    auto dst_md = dnnl::memory::desc({out_dims, dt::f32, tag::nc});
+
+    // Dense description.
+    auto dense_desc = dnnl::inner_product_forward::desc(dnnl::prop_kind::forward_inference, data_md,
+                                                        weight_md, bias_md, dst_md);
+    auto dense_prim_desc = dnnl::inner_product_forward::primitive_desc(dense_desc, engine_);
+
+    auto dense = dnnl::inner_product_forward(dense_prim_desc);
+    net_.push_back(dense);
+
+    // Memories.
+    auto data_memory = BindDNNLMemory(data_entry, data_md);
+    auto weight_memory = BindDNNLMemory(weight_entry, weight_md);
+    auto bias_memory = dnnl::memory(bias_md, engine_);
+    float bias[OC] = {0};
+    write_to_dnnl_memory(bias, bias_memory, OC * sizeof(float));
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto dst_memory = BindDNNLMemory(out_entry, dense_prim_desc.dst_desc());
+
+    net_args_.push_back({{DNNL_ARG_SRC, data_memory},
+                         {DNNL_ARG_WEIGHTS, weight_memory},
+                         {DNNL_ARG_BIAS, bias_memory},
+                         {DNNL_ARG_DST, dst_memory}});
+  }
+
+  void BatchNorm(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    auto data_entry = node.GetInputs()[0];
+    auto gamma_entry = node.GetInputs()[1];
+    auto beta_entry = node.GetInputs()[2];
+    auto mean_entry = node.GetInputs()[3];
+    auto variance_entry = node.GetInputs()[4];
+    dnnl::memory::dims data_shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    dnnl::memory::dim IC = data_shape[1];
+    float epsilon = std::stof(node.GetAttr<std::vector<std::string>>("epsilon")[0]);
+
+    // Memory description.
+    dnnl::memory::desc data_md = GenDNNLMemDescByShape(data_shape, dt::f32);
+
+    // BN description.
+    auto bn_desc = dnnl::batch_normalization_forward::desc(
+        dnnl::prop_kind::forward_inference, data_md, epsilon,
+        dnnl::normalization_flags::use_global_stats | dnnl::normalization_flags::use_scale_shift);
+    auto bn_prim_desc = dnnl::batch_normalization_forward::primitive_desc(bn_desc, engine_);
+    auto bn = dnnl::batch_normalization_forward(bn_prim_desc);
+    net_.push_back(bn);
+
+    // Memories.
+    auto data_memory = BindDNNLMemory(data_entry, data_md);
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto out_memory = BindDNNLMemory(out_entry, data_md);
+    auto mean_memory = BindDNNLMemory(mean_entry, bn_prim_desc.mean_desc());
+    auto variance_memory = BindDNNLMemory(variance_entry, bn_prim_desc.variance_desc());
+
+    // In DNNL, weight is composed of gamma+beta, so we point them to the same DNNL memory but
+    // assign an offset to beta data for runtime serialization.
+    auto weight_memory = BindDNNLMemory(gamma_entry, bn_prim_desc.weights_desc(), 0);
+    BindDNNLMemory(beta_entry, weight_memory, IC);
+
+    net_args_.push_back({{DNNL_ARG_SRC, data_memory},
+                         {DNNL_ARG_DST, out_memory},
+                         {DNNL_ARG_SCALE_SHIFT, weight_memory},
+                         {DNNL_ARG_MEAN, mean_memory},
+                         {DNNL_ARG_VARIANCE, variance_memory}});
+  }
+
+  void Relu(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    auto data_entry = node.GetInputs()[0];
+    dnnl::memory::dims shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    auto data_md = dnnl::memory::desc{{shape}, dt::f32, tag::abcd};
+
+    auto relu_desc = dnnl::eltwise_forward::desc(dnnl::prop_kind::forward_inference,
+                                                 dnnl::algorithm::eltwise_relu, data_md, 0);
+    auto relu_prim_desc = dnnl::eltwise_forward::primitive_desc(relu_desc, engine_);
+    CHECK(data_md == relu_prim_desc.dst_desc());
+
+    auto relu = dnnl::eltwise_forward(relu_prim_desc);
+    net_.push_back(relu);
+
+    auto data_memory = BindDNNLMemory(data_entry, data_md);
+    auto out_md = dnnl::memory::desc(shape, dt::f32, tag::abcd);
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto out_memory = BindDNNLMemory(out_entry, out_md);
+
+    net_args_.push_back({{DNNL_ARG_SRC, data_memory}, {DNNL_ARG_DST, out_memory}});
+  }
+
+  void Add(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    // Memory and compute description.
+    std::vector<dnnl::memory::dims> data_dims;
+    std::vector<dnnl::memory::desc> data_mds;
+    std::vector<dnnl::memory> data_memories;
+
+    CHECK_EQ(node.GetInputs().size(), 2U);
+    for (auto entry : node.GetInputs()) {
+      auto data_shape = nodes_[entry.id_].GetOpShape()[entry.index_];
+      dnnl::memory::desc data_md = GenDNNLMemDescByShape(data_shape, dt::f32);
+
+      data_dims.push_back(data_shape);
+      data_mds.push_back(data_md);
+      data_memories.push_back(BindDNNLMemory(entry, data_md));
+    }
+    CHECK(data_dims[0] == data_dims[1]);
+    auto out_md = data_mds[0];
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto out_memory = BindDNNLMemory(out_entry, out_md);
+
+    auto add_desc =
+        dnnl::binary::desc(dnnl::algorithm::binary_add, data_mds[0], data_mds[1], out_md);
+    auto add_prim_desc = dnnl::binary::primitive_desc(add_desc, engine_);
+    auto add = dnnl::binary(add_prim_desc);
+    net_.push_back(add);
+
+    net_args_.push_back({{DNNL_ARG_SRC_0, data_memories[0]},
+                         {DNNL_ARG_SRC_1, data_memories[1]},
+                         {DNNL_ARG_DST, out_memory}});
+  }
+
+  // Read from DNNL memory (+offset) and write to the handle.
+  inline void read_from_dnnl_memory(void* handle, const dnnl::memory& mem, size_t size,
+                                    size_t offset = 0) {
+    uint8_t* src = static_cast<uint8_t*>(mem.get_data_handle());
+    std::copy(src + offset, src + offset + size, static_cast<uint8_t*>(handle));
+  }
+
+  // Read from the handle and write to DNNL memory (+offset).
+  inline void write_to_dnnl_memory(void* handle, const dnnl::memory& mem, size_t size,
+                                   size_t offset = 0) {
+    uint8_t* dst = static_cast<uint8_t*>(mem.get_data_handle());
+    std::copy(reinterpret_cast<uint8_t*>(handle), reinterpret_cast<uint8_t*>(handle) + size,
+              dst + offset);
+  }
+
+  // Generate DNNL memory description and infer the data layout by the given shape.
+  inline dnnl::memory::desc GenDNNLMemDescByShape(const dnnl::memory::dims& shape, dt dtype) {
+    dnnl::memory::desc data_md;
+    switch (shape.size()) {
+      case 2:
+        data_md = dnnl::memory::desc({shape, dtype, tag::ab});
+        break;
+      case 3:
+        data_md = dnnl::memory::desc({shape, dtype, tag::abc});
+        break;
+      case 4:
+        data_md = dnnl::memory::desc({shape, dtype, tag::abcd});
+        break;
+      case 5:
+        data_md = dnnl::memory::desc({shape, dtype, tag::abcde});
+        break;
+      default:
+        LOG(FATAL) << "Unsupported data shape dimension: " << shape.size();
+        break;
+    }
+    return data_md;
+  }
+
+  /* The dnnl engine. */
+  dnnl::engine engine_;
+  /* The dnnl stream. */
+  dnnl::stream stream_;
+  /* The network layers that are represented in dnnl primitives. */
+  std::vector<dnnl::primitive> net_;
+  /* The memory that is consumed by arguments. */
+  std::vector<std::unordered_map<int, dnnl::memory>> net_args_;
+  /* The entry ID to its corresponding output memory. */
+  std::unordered_map<uint32_t, std::pair<dnnl::memory, size_t>> entry_out_mem_;
+};
+
+runtime::Module DNNLJSONRuntimeCreate(String symbol_name, String graph_json,
+                                      const Array<String>& const_names) {
+  auto n = make_object<DNNLJSONRuntime>(symbol_name.operator std::string(),

Review comment:
       I changed the ones in this PR. We could file another PR to remove the rest in the code base.




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
comaniac commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-653202171


   Wait for #5985.


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

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



[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: tests/python/relay/test_json_runtime.py
##########
@@ -0,0 +1,625 @@
+# 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.
+"""Unit tests for JSON codegen and runtime."""

Review comment:
       Yeah, I think this is actually a broader issue in TVM in that we don't seem to have a robust C++ unit testing framework so a lot of things are tested indirectly via Python. I'll have a think about this, but not blocking on 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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();
+};
+
+/*! \brief Serialize a Relay expression to JSON. */
+class JSONSerializer : public MemoizedExprTranslator<std::vector<JSONGraphNodeEntry>> {
+ public:
+  /*!
+   * \brief Constructor
+   *
+   * \param symbol The symbol that represents the graph being converted.
+   * \param expr The Relay expression to be converted to the JSON form.
+   */
+  JSONSerializer(const std::string& symbol, const Expr& expr) : symbol_(symbol), func_(expr) {}
+
+  void serialize() {
+    relay::Function func = Downcast<relay::Function>(func_);
+    // First we convert all the parameters into input nodes.
+    for (const auto& param : func->params) {
+      auto node_ptr = std::make_shared<JSONGraphNode>(param->name_hint(), "input" /* op_type_ */);
+      memo_[param] = AddNode(node_ptr, param);
+    }
+    heads_ = VisitExpr(func->body);
+  }
+
+  /*!\brief Return the required params. */
+  Array<String> GetParams() const { return params_; }
+
+  /*!\brief Return the generated json. */
+  std::string GetJSON() {
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    Save(&writer);
+    return os.str();
+  }
+
+ protected:
+  /*!
+   * \brief Add a node to graph.
+   *
+   * \param node A graph node. It is a shared pointer. Some attributes of it
+   *        will be added, i.e. shape and type. These attributes are attached to
+   *        the JSON graph in the end.
+   * \param expr The relay expression.
+   * \return A list of graph entry nodes. It the relay expr is a tuple type, we
+   *         will flatten it.
+   */
+  std::vector<JSONGraphNodeEntry> AddNode(JSONGraphObjectPtr node, const Expr& expr) {
+    auto checked_type = expr->checked_type();
+    auto node_id = nodes_.size();
+    nodes_.push_back(node);
+    std::vector<JSONGraphNodeEntry> ret;
+    ShapeVector shape;
+    TypeVector dtype;
+    // Flatten tuple node.
+    if (const auto* tuple_type = checked_type.as<TupleTypeNode>()) {
+      for (size_t i = 0; i < tuple_type->fields.size(); ++i) {
+        const auto* tensor_type = tuple_type->fields[i].as<TensorTypeNode>();
+        CHECK(tensor_type) << "Expect TensorType, but received: ."
+                           << tuple_type->fields[i]->GetTypeKey();
+        ret.push_back(JSONGraphNodeEntry(node_id, i));
+        shape.emplace_back(GetIntShape(tensor_type->shape));
+        dtype.emplace_back(DType2String(tensor_type->dtype));
+      }
+      node->SetNumOutput(tuple_type->fields.size());
+    } else {
+      const auto* tensor_type = checked_type.as<TensorTypeNode>();
+      CHECK(tensor_type) << "Expect TensorType, but received: ." << checked_type->GetTypeKey();
+      shape.emplace_back(GetIntShape(tensor_type->shape));
+      dtype.emplace_back(DType2String(tensor_type->dtype));
+      ret.push_back(JSONGraphNodeEntry(node_id, 0));
+    }
+    std::vector<dmlc::any> shape_attrs;
+    shape_attrs.emplace_back(shape);
+    node->SetAttr("shape", shape_attrs);
+
+    std::vector<dmlc::any> type_attrs;
+    type_attrs.emplace_back(dtype);
+    node->SetAttr("dtype", type_attrs);
+    return ret;
+  }
+
+  void SetCallNodeAttribute(JSONGraphObjectPtr node, const CallNode* cn) {
+    if (cn->op.as<OpNode>()) {
+      OpAttrExtractor extractor(node);
+      const Object* call_attr = cn->attrs.get();
+      extractor.Extract(const_cast<Object*>(call_attr));
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto pattern = fn->GetAttr<String>(attr::kPartitionedFromPattern);
+      CHECK(pattern.defined());
+      std::vector<std::string> values;
+      values.push_back(pattern.value().operator std::string());
+      std::vector<dmlc::any> attr;
+      attr.emplace_back(values);
+      node->SetAttr("PartitionedFromPattern", attr);

Review comment:
       Composite is set as the op name of that CallNode: https://github.com/apache/incubator-tvm/pull/5919/files#diff-18856f7025844f75dba4bd31f5be251aR271-R274




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

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



[GitHub] [incubator-tvm] lhutton1 commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       Thanks all for looking into this, using the latest commit worked




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       I also hit this error
   
   ```
   Traceback (most recent call last):
     File "test_pass_partition_graph.py", line 1294, in <module>
       test_extern_dnnl()
     File "test_pass_partition_graph.py", line 455, in test_extern_dnnl
       (1, 32, 14, 14), ref_res.asnumpy(), tol=1e-5)
     File "test_pass_partition_graph.py", line 231, in check_result
       check_vm_result()
     File "test_pass_partition_graph.py", line 199, in check_vm_result
       exe = relay.vm.compile(mod, target=target, params=params)
     File "/home/masa/projects/dev/tvm/python/tvm/relay/backend/vm.py", line 70, in compile
       compiler.codegen()
     File "/home/masa/projects/dev/tvm/python/tvm/relay/backend/vm.py", line 139, in codegen
       self._codegen()
     File "/home/masa/projects/dev/tvm/python/tvm/_ffi/_ctypes/packed_func.py", line 225, in __call__
       raise get_last_ffi_error()
   tvm._ffi.base.TVMError: Traceback (most recent call last):
     [bt] (8) /home/masa/projects/dev/tvm/build/libtvm.so(TVMFuncCall+0x5f) [0x7f211cc4bbcf]
     [bt] (7) /home/masa/projects/dev/tvm/build/libtvm.so(+0xda8ae9) [0x7f211cb11ae9]
     [bt] (6) /home/masa/projects/dev/tvm/build/libtvm.so(tvm::relay::vm::VMCompiler::Codegen()+0x628) [0x7f211cb10fe8]
     [bt] (5) /home/masa/projects/dev/tvm/build/libtvm.so(tvm::relay::CompileEngineImpl::LowerExternalFunctions()+0x6f7) [0x7f211cac10c7]
     [bt] (4) /home/masa/projects/dev/tvm/build/libtvm.so(tvm::runtime::TypedPackedFunc<tvm::runtime::Module (tvm::runtime::ObjectRef const&)>::AssignTypedLambda<tvm::runtime::Module (*)(tvm::runtime::ObjectRef const&)>(tvm::runtime::Module (*)(tvm::runtime::ObjectRef const&))::{lambda(tvm::runtime::TVMArgs const&, tvm::runtime::TVMRetValue*)#1}::operator()(tvm::runtime::TVMArgs const&, tvm::runtime::TVMRetValue*) const+0xf8) [0x7f211cc25238]
     [bt] (3) /home/masa/projects/dev/tvm/build/libtvm.so(tvm::relay::contrib::DNNLCompiler(tvm::runtime::ObjectRef const&)+0xa52) [0x7f211cc2c572]
     [bt] (2) /home/masa/projects/dev/tvm/build/libtvm.so(tvm::relay::backend::contrib::JSONSerializer::Save(dmlc::JSONWriter*)+0x58e) [0x7f211cc32eee]
     [bt] (1) /home/masa/projects/dev/tvm/build/libtvm.so(tvm::runtime::json::JSONGraphNode::Save(dmlc::JSONWriter*)+0x9f3) [0x7f211cc32153]
     [bt] (0) /home/masa/projects/dev/tvm/build/libtvm.so(+0xec2938) [0x7f211cc2b938]
     File "/home/masa/projects/dev/tvm/3rdparty/dmlc-core/include/dmlc/././json.h", line 590
   TVMError: Check failed: it != nmap.end() && it->first == id: Type St6vectorIN4dmlc3anyESaIS1_EE has not been registered via DMLC_JSON_ENABLE_ANY
   ```




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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       It is not about current handler, it is about the specialization of `struct Handler<std::vector<dmlc::any>>`




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

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



[GitHub] [incubator-tvm] lhutton1 commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
lhutton1 commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-651272256


   > > Possibly out of scope for this PR but is there a plan to support multiple functions/sub-graphs? Currently it looks like there is only support for a single dnnl sub-graph after the graph is partitioned?
   > 
   > We now have only one subgraph per module, but we could have many modules to support multiple subgraphs. Please see @mbaret 's comments to this PR and the discussions for details.
   
   Apologies, missed that, 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



[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();
+};
+
+/*! \brief Serialize a Relay expression to JSON. */
+class JSONSerializer : public MemoizedExprTranslator<std::vector<JSONGraphNodeEntry>> {
+ public:
+  /*!
+   * \brief Constructor
+   *
+   * \param symbol The symbol that represents the graph being converted.
+   * \param expr The Relay expression to be converted to the JSON form.
+   */
+  JSONSerializer(const std::string& symbol, const Expr& expr) : symbol_(symbol), func_(expr) {}
+
+  void serialize() {
+    relay::Function func = Downcast<relay::Function>(func_);
+    // First we convert all the parameters into input nodes.
+    for (const auto& param : func->params) {
+      auto node_ptr = std::make_shared<JSONGraphNode>(param->name_hint(), "input" /* op_type_ */);
+      memo_[param] = AddNode(node_ptr, param);
+    }
+    heads_ = VisitExpr(func->body);
+  }
+
+  /*!\brief Return the required params. */
+  Array<String> GetParams() const { return params_; }
+
+  /*!\brief Return the generated json. */
+  std::string GetJSON() {
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    Save(&writer);
+    return os.str();
+  }
+
+ protected:
+  /*!
+   * \brief Add a node to graph.
+   *
+   * \param node A graph node. It is a shared pointer. Some attributes of it
+   *        will be added, i.e. shape and type. These attributes are attached to
+   *        the JSON graph in the end.
+   * \param expr The relay expression.
+   * \return A list of graph entry nodes. It the relay expr is a tuple type, we
+   *         will flatten it.
+   */
+  std::vector<JSONGraphNodeEntry> AddNode(JSONGraphObjectPtr node, const Expr& expr) {
+    auto checked_type = expr->checked_type();
+    auto node_id = nodes_.size();
+    nodes_.push_back(node);
+    std::vector<JSONGraphNodeEntry> ret;
+    ShapeVector shape;
+    TypeVector dtype;
+    // Flatten tuple node.
+    if (const auto* tuple_type = checked_type.as<TupleTypeNode>()) {
+      for (size_t i = 0; i < tuple_type->fields.size(); ++i) {
+        const auto* tensor_type = tuple_type->fields[i].as<TensorTypeNode>();
+        CHECK(tensor_type) << "Expect TensorType, but received: ."
+                           << tuple_type->fields[i]->GetTypeKey();
+        ret.push_back(JSONGraphNodeEntry(node_id, i));
+        shape.emplace_back(GetIntShape(tensor_type->shape));
+        dtype.emplace_back(DType2String(tensor_type->dtype));
+      }
+      node->SetNumOutput(tuple_type->fields.size());
+    } else {
+      const auto* tensor_type = checked_type.as<TensorTypeNode>();
+      CHECK(tensor_type) << "Expect TensorType, but received: ." << checked_type->GetTypeKey();
+      shape.emplace_back(GetIntShape(tensor_type->shape));
+      dtype.emplace_back(DType2String(tensor_type->dtype));
+      ret.push_back(JSONGraphNodeEntry(node_id, 0));
+    }
+    std::vector<dmlc::any> shape_attrs;
+    shape_attrs.emplace_back(shape);
+    node->SetAttr("shape", shape_attrs);
+
+    std::vector<dmlc::any> type_attrs;
+    type_attrs.emplace_back(dtype);
+    node->SetAttr("dtype", type_attrs);
+    return ret;
+  }
+
+  void SetCallNodeAttribute(JSONGraphObjectPtr node, const CallNode* cn) {
+    if (cn->op.as<OpNode>()) {
+      OpAttrExtractor extractor(node);
+      const Object* call_attr = cn->attrs.get();
+      extractor.Extract(const_cast<Object*>(call_attr));
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto pattern = fn->GetAttr<String>(attr::kPartitionedFromPattern);
+      CHECK(pattern.defined());
+      std::vector<std::string> values;
+      values.push_back(pattern.value().operator std::string());
+      std::vector<dmlc::any> attr;
+      attr.emplace_back(values);
+      node->SetAttr("PartitionedFromPattern", attr);

Review comment:
       Ah yes :)




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       The seems to be about `std::vector`, so I don't think `Handler<std::unordered_map<std::string, dmlc::any>>` is related?




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

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



[GitHub] [incubator-tvm] lhutton1 commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
lhutton1 commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-651268600


   Possibly out of scope for this PR but is there a plan to support multiple functions/sub-graphs? Currently it looks like there is only support for a single dnnl sub-graph?


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});

Review comment:
       Hmm...I think we did this to simplify the attribute deserialization process. Otherwise, we need to figure out an approach to get the actual attribute type. @zhiics may comment more on 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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/graph_runtime_codegen.cc
##########
@@ -661,6 +660,8 @@ struct Handler<std::unordered_map<std::string, dmlc::any>> {
         writer->WriteObjectKeyValue(k, dmlc::get<std::vector<std::vector<int64_t>>>(v));
       } else if (SameType<std::vector<std::string>>(v)) {
         writer->WriteObjectKeyValue(k, dmlc::get<std::vector<std::string>>(v));
+      } else if (SameType<std::vector<dmlc::any>>(v)) {

Review comment:
       Thanks for catching on this. This change supports any type of vectors to be serialized and we need a corresponding process in `json_node.h`. I'll push a commit to fix 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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       I launched a new instance to run the unit tests but still didn't encounter this issue. Maybe you can clone a new one and try again? We will also work on enabling DNNL on the CI so that we can see if this is an environment issue.




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: tests/python/relay/test_json_runtime.py
##########
@@ -0,0 +1,625 @@
+# 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.
+"""Unit tests for JSON codegen and runtime."""

Review comment:
       The case you mentioned is indeed more desired, but that means we have to enhance the example c compiler to support all these ops, which seems impractical. That's why enabling DNNL in the CI is important.




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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       yeah




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

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



[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/graph_runtime_codegen.cc
##########
@@ -661,6 +660,8 @@ struct Handler<std::unordered_map<std::string, dmlc::any>> {
         writer->WriteObjectKeyValue(k, dmlc::get<std::vector<std::vector<int64_t>>>(v));
       } else if (SameType<std::vector<std::string>>(v)) {
         writer->WriteObjectKeyValue(k, dmlc::get<std::vector<std::string>>(v));
+      } else if (SameType<std::vector<dmlc::any>>(v)) {

Review comment:
       What does this change do?

##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();
+};
+
+/*! \brief Serialize a Relay expression to JSON. */
+class JSONSerializer : public MemoizedExprTranslator<std::vector<JSONGraphNodeEntry>> {
+ public:
+  /*!
+   * \brief Constructor
+   *
+   * \param symbol The symbol that represents the graph being converted.
+   * \param expr The Relay expression to be converted to the JSON form.
+   */
+  JSONSerializer(const std::string& symbol, const Expr& expr) : symbol_(symbol), func_(expr) {}
+
+  void serialize() {
+    relay::Function func = Downcast<relay::Function>(func_);
+    // First we convert all the parameters into input nodes.
+    for (const auto& param : func->params) {
+      auto node_ptr = std::make_shared<JSONGraphNode>(param->name_hint(), "input" /* op_type_ */);
+      memo_[param] = AddNode(node_ptr, param);
+    }
+    heads_ = VisitExpr(func->body);
+  }
+
+  /*!\brief Return the required params. */
+  Array<String> GetParams() const { return params_; }
+
+  /*!\brief Return the generated json. */
+  std::string GetJSON() {
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    Save(&writer);
+    return os.str();
+  }
+
+ protected:
+  /*!
+   * \brief Add a node to graph.
+   *
+   * \param node A graph node. It is a shared pointer. Some attributes of it
+   *        will be added, i.e. shape and type. These attributes are attached to
+   *        the JSON graph in the end.
+   * \param expr The relay expression.
+   * \return A list of graph entry nodes. It the relay expr is a tuple type, we
+   *         will flatten it.
+   */
+  std::vector<JSONGraphNodeEntry> AddNode(JSONGraphObjectPtr node, const Expr& expr) {
+    auto checked_type = expr->checked_type();
+    auto node_id = nodes_.size();
+    nodes_.push_back(node);
+    std::vector<JSONGraphNodeEntry> ret;
+    ShapeVector shape;
+    TypeVector dtype;
+    // Flatten tuple node.
+    if (const auto* tuple_type = checked_type.as<TupleTypeNode>()) {
+      for (size_t i = 0; i < tuple_type->fields.size(); ++i) {
+        const auto* tensor_type = tuple_type->fields[i].as<TensorTypeNode>();
+        CHECK(tensor_type) << "Expect TensorType, but received: ."
+                           << tuple_type->fields[i]->GetTypeKey();
+        ret.push_back(JSONGraphNodeEntry(node_id, i));
+        shape.emplace_back(GetIntShape(tensor_type->shape));
+        dtype.emplace_back(DType2String(tensor_type->dtype));
+      }
+      node->SetNumOutput(tuple_type->fields.size());
+    } else {
+      const auto* tensor_type = checked_type.as<TensorTypeNode>();
+      CHECK(tensor_type) << "Expect TensorType, but received: ." << checked_type->GetTypeKey();

Review comment:
       "." in the wrong place?

##########
File path: src/runtime/contrib/json/json_runtime.h
##########
@@ -0,0 +1,267 @@
+/*
+ * 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/json/json_runtime.h
+ * \brief Utilities for json runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/ndarray.h>
+
+#include <cstddef>
+#include <string>
+#include <tuple>
+#include <type_traits>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "json_node.h"
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+/*!
+ * \brief A json runtime that executes the serialized JSON format. This runtime
+ * can be extended by user defined runtime for execution.
+ */
+class JSONRuntimeBase : public ModuleNode {
+ public:
+  JSONRuntimeBase(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : symbol_name_(symbol_name), graph_json_(graph_json), const_names_(const_names) {
+    LoadGraph(graph_json_);
+  }
+
+  const char* type_key() const { return "json"; }
+
+  /*! \brief Initialize a specific json runtime. */
+  virtual void Init(const Array<NDArray>& consts) = 0;
+
+  /*! \brief Invoke the execution engine to inteprete a specific json runtime. */
+  virtual void Run() = 0;
+
+  /*!
+   * \brief Get a packed function.
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK_EQ(args.size(), 1U);
+        this->Init(args[0]);
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  virtual void SaveToBinary(dmlc::Stream* stream) {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> consts;
+    for (const auto& it : const_names_) {
+      consts.push_back(it);
+    }
+    stream->Write(consts);
+  }
+
+  template <typename T,
+            typename = typename std::enable_if<std::is_base_of<JSONRuntimeBase, T>::value>::type>
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    auto n = make_object<T>(symbol, graph_json, const_names);
+    return Module(n);
+  }
+
+ protected:
+  /*!
+   * \brief Set up the input and output buffers by binding their DLTensor pointers to the
+   * corresponding data entry.
+   *
+   * \param args The packed args.
+   */
+  void SetInputOutputBuffers(const TVMArgs& args) {
+    CHECK_EQ(args.size(), input_var_idx_.size() + outputs_.size())
+        << "Found mismatch in the number of provided data entryies and required.";
+
+    for (size_t i = 0; i < static_cast<size_t>(args.size()); i++) {
+      auto eid = i < input_var_idx_.size() ? EntryID(input_var_idx_[i], 0)
+                                           : EntryID(outputs_[i - input_var_idx_.size()]);
+      CHECK(args[i].type_code() == kTVMNDArrayHandle || args[i].type_code() == kTVMDLTensorHandle)
+          << "Expect NDArray or DLTensor as inputs";
+
+      const DLTensor* arg;
+      if (args[i].IsObjectRef<NDArray>()) {
+        NDArray arr = args[i];
+        arg = arr.operator->();
+      } else {
+        arg = args[i].operator DLTensor*();
+      }
+
+      // Assign input/output the NDArray pointers to data entry so that we can directly
+      // read/write host buffers.
+      data_entry_[eid] = arg;
+    }
+  }
+
+  /*!
+   * \brief Load the graph and record the entries for inputs and constants.
+   *
+   * \param graph_json The graph in the json format.
+   */
+  void LoadGraph(const std::string& graph_json) {
+    std::istringstream is(graph_json);
+    dmlc::JSONReader reader(&is);
+    this->Load(&reader);
+    std::vector<std::string> consts;
+    for (size_t i = 0; i < input_nodes_.size(); i++) {
+      uint32_t nid = input_nodes_[i];
+      std::string name = nodes_[nid].name_;
+      if (nodes_[nid].op_type_ == "input") {
+        input_var_idx_.push_back(nid);
+      } else {
+        CHECK_EQ(nodes_[nid].op_type_, "const");
+        auto pos = std::find(std::begin(const_names_), std::end(const_names_), name);
+        CHECK(pos != std::end(const_names_)) << "Found non-existent constant: " << name;
+        const_idx_.push_back(nid);
+        consts.push_back(name);
+      }
+    }
+    CHECK_EQ(consts.size(), const_names_.size())
+        << "Found mismatch for the number of constants in the graph and required.";
+
+    for (size_t i = 0; i < consts.size(); i++) {
+      CHECK_EQ(consts[i], const_names_[i])
+          << "The position of constant in the graph must be the same as the required.";
+    }
+
+    // Reserve data entries.
+    data_entry_.resize(NumEntries());
+  }
+
+  /*!
+   * \brief Set up the constants/weights for inference by binding their DLTensor pointer to
+   * the corresponding data entry.
+   *
+   * \param consts A list of constant NDArray to be used.
+   */
+  void SetupConstants(const Array<NDArray>& consts) {
+    for (size_t i = 0; i < consts.size(); ++i) {
+      data_entry_[const_idx_[i]] = consts[i].operator->();
+    }
+  }
+
+  // Load the graph.
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "nodes") {
+        reader->Read(&nodes_);
+      } else if (key == "arg_nodes") {
+        reader->Read(&input_nodes_);
+      } else if (key == "node_row_ptr") {
+        reader->Read(&node_row_ptr_);
+      } else if (key == "heads") {
+        reader->Read(&outputs_);
+      } else {
+        LOG(FATAL) << "Unknow key: " << key;
+      }
+    }
+  }
+
+  // Get the node entry index.
+  uint32_t EntryID(uint32_t nid, uint32_t index) const { return node_row_ptr_[nid] + index; }
+
+  // Get the node entry index.
+  uint32_t EntryID(const JSONGraphNodeEntry& e) const { return EntryID(e.id_, e.index_); }
+
+  // Number of node entries.
+  uint32_t NumEntries() const { return node_row_ptr_.back(); }
+
+ protected:
+  /* The only subgraph name for this module. */
+  std::string symbol_name_;
+  /* The graph. */
+  std::string graph_json_;
+  /* The required constant names. */

Review comment:
       Should these have \brief too?

##########
File path: src/runtime/contrib/json/json_runtime.h
##########
@@ -0,0 +1,267 @@
+/*
+ * 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/json/json_runtime.h
+ * \brief Utilities for json runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/ndarray.h>
+
+#include <cstddef>
+#include <string>
+#include <tuple>
+#include <type_traits>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "json_node.h"
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+/*!
+ * \brief A json runtime that executes the serialized JSON format. This runtime
+ * can be extended by user defined runtime for execution.
+ */
+class JSONRuntimeBase : public ModuleNode {
+ public:
+  JSONRuntimeBase(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : symbol_name_(symbol_name), graph_json_(graph_json), const_names_(const_names) {
+    LoadGraph(graph_json_);
+  }
+
+  const char* type_key() const { return "json"; }
+
+  /*! \brief Initialize a specific json runtime. */
+  virtual void Init(const Array<NDArray>& consts) = 0;
+
+  /*! \brief Invoke the execution engine to inteprete a specific json runtime. */
+  virtual void Run() = 0;
+
+  /*!
+   * \brief Get a packed function.
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK_EQ(args.size(), 1U);
+        this->Init(args[0]);
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  virtual void SaveToBinary(dmlc::Stream* stream) {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> consts;
+    for (const auto& it : const_names_) {
+      consts.push_back(it);
+    }
+    stream->Write(consts);
+  }
+
+  template <typename T,
+            typename = typename std::enable_if<std::is_base_of<JSONRuntimeBase, T>::value>::type>
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";
+    CHECK(stream->Read(&graph_json)) << "Loading graph json failed";
+    CHECK(stream->Read(&consts)) << "Loading the const name list failed";
+    Array<String> const_names;
+    for (const auto& it : consts) {
+      const_names.push_back(it);
+    }
+    auto n = make_object<T>(symbol, graph_json, const_names);
+    return Module(n);
+  }
+
+ protected:
+  /*!
+   * \brief Set up the input and output buffers by binding their DLTensor pointers to the
+   * corresponding data entry.
+   *
+   * \param args The packed args.
+   */
+  void SetInputOutputBuffers(const TVMArgs& args) {
+    CHECK_EQ(args.size(), input_var_idx_.size() + outputs_.size())
+        << "Found mismatch in the number of provided data entryies and required.";
+
+    for (size_t i = 0; i < static_cast<size_t>(args.size()); i++) {
+      auto eid = i < input_var_idx_.size() ? EntryID(input_var_idx_[i], 0)
+                                           : EntryID(outputs_[i - input_var_idx_.size()]);
+      CHECK(args[i].type_code() == kTVMNDArrayHandle || args[i].type_code() == kTVMDLTensorHandle)
+          << "Expect NDArray or DLTensor as inputs";
+
+      const DLTensor* arg;
+      if (args[i].IsObjectRef<NDArray>()) {
+        NDArray arr = args[i];
+        arg = arr.operator->();
+      } else {
+        arg = args[i].operator DLTensor*();
+      }
+
+      // Assign input/output the NDArray pointers to data entry so that we can directly
+      // read/write host buffers.
+      data_entry_[eid] = arg;
+    }
+  }
+
+  /*!
+   * \brief Load the graph and record the entries for inputs and constants.
+   *
+   * \param graph_json The graph in the json format.
+   */
+  void LoadGraph(const std::string& graph_json) {
+    std::istringstream is(graph_json);
+    dmlc::JSONReader reader(&is);
+    this->Load(&reader);
+    std::vector<std::string> consts;
+    for (size_t i = 0; i < input_nodes_.size(); i++) {
+      uint32_t nid = input_nodes_[i];
+      std::string name = nodes_[nid].name_;
+      if (nodes_[nid].op_type_ == "input") {
+        input_var_idx_.push_back(nid);
+      } else {
+        CHECK_EQ(nodes_[nid].op_type_, "const");
+        auto pos = std::find(std::begin(const_names_), std::end(const_names_), name);
+        CHECK(pos != std::end(const_names_)) << "Found non-existent constant: " << name;
+        const_idx_.push_back(nid);
+        consts.push_back(name);
+      }
+    }
+    CHECK_EQ(consts.size(), const_names_.size())
+        << "Found mismatch for the number of constants in the graph and required.";
+
+    for (size_t i = 0; i < consts.size(); i++) {
+      CHECK_EQ(consts[i], const_names_[i])
+          << "The position of constant in the graph must be the same as the required.";
+    }
+
+    // Reserve data entries.
+    data_entry_.resize(NumEntries());
+  }
+
+  /*!
+   * \brief Set up the constants/weights for inference by binding their DLTensor pointer to
+   * the corresponding data entry.
+   *
+   * \param consts A list of constant NDArray to be used.
+   */
+  void SetupConstants(const Array<NDArray>& consts) {
+    for (size_t i = 0; i < consts.size(); ++i) {
+      data_entry_[const_idx_[i]] = consts[i].operator->();
+    }
+  }
+
+  // Load the graph.
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "nodes") {
+        reader->Read(&nodes_);
+      } else if (key == "arg_nodes") {
+        reader->Read(&input_nodes_);
+      } else if (key == "node_row_ptr") {
+        reader->Read(&node_row_ptr_);
+      } else if (key == "heads") {
+        reader->Read(&outputs_);
+      } else {
+        LOG(FATAL) << "Unknow key: " << key;

Review comment:
       Unknown

##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();
+};
+
+/*! \brief Serialize a Relay expression to JSON. */
+class JSONSerializer : public MemoizedExprTranslator<std::vector<JSONGraphNodeEntry>> {
+ public:
+  /*!
+   * \brief Constructor
+   *
+   * \param symbol The symbol that represents the graph being converted.
+   * \param expr The Relay expression to be converted to the JSON form.
+   */
+  JSONSerializer(const std::string& symbol, const Expr& expr) : symbol_(symbol), func_(expr) {}
+
+  void serialize() {
+    relay::Function func = Downcast<relay::Function>(func_);
+    // First we convert all the parameters into input nodes.
+    for (const auto& param : func->params) {
+      auto node_ptr = std::make_shared<JSONGraphNode>(param->name_hint(), "input" /* op_type_ */);
+      memo_[param] = AddNode(node_ptr, param);
+    }
+    heads_ = VisitExpr(func->body);
+  }
+
+  /*!\brief Return the required params. */
+  Array<String> GetParams() const { return params_; }
+
+  /*!\brief Return the generated json. */
+  std::string GetJSON() {
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    Save(&writer);
+    return os.str();
+  }
+
+ protected:
+  /*!
+   * \brief Add a node to graph.
+   *
+   * \param node A graph node. It is a shared pointer. Some attributes of it
+   *        will be added, i.e. shape and type. These attributes are attached to
+   *        the JSON graph in the end.
+   * \param expr The relay expression.
+   * \return A list of graph entry nodes. It the relay expr is a tuple type, we
+   *         will flatten it.
+   */
+  std::vector<JSONGraphNodeEntry> AddNode(JSONGraphObjectPtr node, const Expr& expr) {
+    auto checked_type = expr->checked_type();
+    auto node_id = nodes_.size();
+    nodes_.push_back(node);
+    std::vector<JSONGraphNodeEntry> ret;
+    ShapeVector shape;
+    TypeVector dtype;
+    // Flatten tuple node.
+    if (const auto* tuple_type = checked_type.as<TupleTypeNode>()) {
+      for (size_t i = 0; i < tuple_type->fields.size(); ++i) {
+        const auto* tensor_type = tuple_type->fields[i].as<TensorTypeNode>();
+        CHECK(tensor_type) << "Expect TensorType, but received: ."
+                           << tuple_type->fields[i]->GetTypeKey();
+        ret.push_back(JSONGraphNodeEntry(node_id, i));
+        shape.emplace_back(GetIntShape(tensor_type->shape));
+        dtype.emplace_back(DType2String(tensor_type->dtype));
+      }
+      node->SetNumOutput(tuple_type->fields.size());
+    } else {
+      const auto* tensor_type = checked_type.as<TensorTypeNode>();
+      CHECK(tensor_type) << "Expect TensorType, but received: ." << checked_type->GetTypeKey();
+      shape.emplace_back(GetIntShape(tensor_type->shape));
+      dtype.emplace_back(DType2String(tensor_type->dtype));
+      ret.push_back(JSONGraphNodeEntry(node_id, 0));
+    }
+    std::vector<dmlc::any> shape_attrs;
+    shape_attrs.emplace_back(shape);
+    node->SetAttr("shape", shape_attrs);
+
+    std::vector<dmlc::any> type_attrs;
+    type_attrs.emplace_back(dtype);
+    node->SetAttr("dtype", type_attrs);
+    return ret;
+  }
+
+  void SetCallNodeAttribute(JSONGraphObjectPtr node, const CallNode* cn) {
+    if (cn->op.as<OpNode>()) {
+      OpAttrExtractor extractor(node);
+      const Object* call_attr = cn->attrs.get();
+      extractor.Extract(const_cast<Object*>(call_attr));
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto pattern = fn->GetAttr<String>(attr::kPartitionedFromPattern);
+      CHECK(pattern.defined());
+      std::vector<std::string> values;
+      values.push_back(pattern.value().operator std::string());
+      std::vector<dmlc::any> attr;
+      attr.emplace_back(values);
+      node->SetAttr("PartitionedFromPattern", attr);

Review comment:
       Composite should also be set.

##########
File path: src/runtime/metadata_module.cc
##########
@@ -48,15 +48,22 @@ class MetadataModuleNode : public ModuleNode {
  public:
   MetadataModuleNode(const std::unordered_map<std::string, NDArray>& metadata,
                      const std::unordered_map<std::string, std::vector<std::string>>& sym_vars)
-      : metadata_(metadata), sym_vars_(sym_vars) {}
+      : metadata_(metadata), sym_vars_(sym_vars) {
+    // Only the related submodules are cached to reduce the number of runtime
+    // symbol lookup for initialization. Otherwise, symbols/primitives in the
+    // DSO module will also be cached but they never need to be initialized.
+    for (const auto& it : sym_vars_) {
+      initialized_[it.first] = false;
+    }
+  }

Review comment:
       These changes (and the others in this file) seem unrelated to this patch.

##########
File path: src/runtime/contrib/json/json_runtime.h
##########
@@ -0,0 +1,267 @@
+/*
+ * 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/json/json_runtime.h

Review comment:
       src/runtime/contrib/json/json_runtime.h

##########
File path: tests/python/relay/test_json_runtime.py
##########
@@ -0,0 +1,625 @@
+# 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.
+"""Unit tests for JSON codegen and runtime."""

Review comment:
       This seems more like it's testing the DNNL codegen and runtime rather than the underlying JSON implementation, to the point no tests will run without DNNL. I think we should aim to steer away from this sort of approach and see if there are ways we can more directly test the implementation.

##########
File path: src/runtime/contrib/json/json_runtime.h
##########
@@ -0,0 +1,267 @@
+/*
+ * 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/json/json_runtime.h
+ * \brief Utilities for json runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/ndarray.h>
+
+#include <cstddef>
+#include <string>
+#include <tuple>
+#include <type_traits>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "json_node.h"
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+/*!
+ * \brief A json runtime that executes the serialized JSON format. This runtime
+ * can be extended by user defined runtime for execution.
+ */
+class JSONRuntimeBase : public ModuleNode {
+ public:
+  JSONRuntimeBase(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : symbol_name_(symbol_name), graph_json_(graph_json), const_names_(const_names) {
+    LoadGraph(graph_json_);
+  }
+
+  const char* type_key() const { return "json"; }
+
+  /*! \brief Initialize a specific json runtime. */
+  virtual void Init(const Array<NDArray>& consts) = 0;
+
+  /*! \brief Invoke the execution engine to inteprete a specific json runtime. */
+  virtual void Run() = 0;
+
+  /*!
+   * \brief Get a packed function.
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK_EQ(args.size(), 1U);
+        this->Init(args[0]);
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  virtual void SaveToBinary(dmlc::Stream* stream) {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> consts;
+    for (const auto& it : const_names_) {
+      consts.push_back(it);
+    }
+    stream->Write(consts);
+  }
+
+  template <typename T,
+            typename = typename std::enable_if<std::is_base_of<JSONRuntimeBase, T>::value>::type>
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";

Review comment:
       This implies the module can only hold a single function, and I'm assuming this was the motivation behind the broader changes to the compiler flow to move from IRModule -> Function for external codegen. Are there any downsides/overheads to only allowing single function modules?




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_runtime.h
##########
@@ -0,0 +1,267 @@
+/*
+ * 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/json/json_runtime.h
+ * \brief Utilities for json runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/ndarray.h>
+
+#include <cstddef>
+#include <string>
+#include <tuple>
+#include <type_traits>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "json_node.h"
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+/*!
+ * \brief A json runtime that executes the serialized JSON format. This runtime
+ * can be extended by user defined runtime for execution.
+ */
+class JSONRuntimeBase : public ModuleNode {
+ public:
+  JSONRuntimeBase(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : symbol_name_(symbol_name), graph_json_(graph_json), const_names_(const_names) {
+    LoadGraph(graph_json_);
+  }
+
+  const char* type_key() const { return "json"; }
+
+  /*! \brief Initialize a specific json runtime. */
+  virtual void Init(const Array<NDArray>& consts) = 0;
+
+  /*! \brief Invoke the execution engine to inteprete a specific json runtime. */
+  virtual void Run() = 0;
+
+  /*!
+   * \brief Get a packed function.
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK_EQ(args.size(), 1U);
+        this->Init(args[0]);
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  virtual void SaveToBinary(dmlc::Stream* stream) {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> consts;
+    for (const auto& it : const_names_) {
+      consts.push_back(it);
+    }
+    stream->Write(consts);
+  }
+
+  template <typename T,
+            typename = typename std::enable_if<std::is_base_of<JSONRuntimeBase, T>::value>::type>
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";

Review comment:
       We had a discussion with @tqchen and our conclusion was that the overhead is ignorable. Looking into the implementation, the module initialization process is required for each subgraph, and the potential overhead of copying shared tensors from one metadata module to many customized modules has been avoided by zero-copy.




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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       Can't reproduce the issue on my local. I'll use another instance to try again. Meanwhile, could you check if you have run `git submodule update` to update the dmlc dependency?




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});

Review comment:
       remove `operator std::string()`




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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();

Review comment:
       `reflection_` is the ReflectionTable that helps us visit the attributes. 
   
   https://github.com/apache/incubator-tvm/blob/44194d017d28643ab1eb54044d2a1d847ac9f4c8/include/tvm/node/reflection.h#L81
   
   `Extract` is the method to extract the attribute of an operator, like strides and padding in convolution.




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();
+};
+
+/*! \brief Serialize a Relay expression to JSON. */
+class JSONSerializer : public MemoizedExprTranslator<std::vector<JSONGraphNodeEntry>> {
+ public:
+  /*!
+   * \brief Constructor
+   *
+   * \param symbol The symbol that represents the graph being converted.
+   * \param expr The Relay expression to be converted to the JSON form.
+   */
+  JSONSerializer(const std::string& symbol, const Expr& expr) : symbol_(symbol), func_(expr) {}
+
+  void serialize() {
+    relay::Function func = Downcast<relay::Function>(func_);
+    // First we convert all the parameters into input nodes.
+    for (const auto& param : func->params) {
+      auto node_ptr = std::make_shared<JSONGraphNode>(param->name_hint(), "input" /* op_type_ */);
+      memo_[param] = AddNode(node_ptr, param);
+    }
+    heads_ = VisitExpr(func->body);
+  }
+
+  /*!\brief Return the required params. */
+  Array<String> GetParams() const { return params_; }
+
+  /*!\brief Return the generated json. */
+  std::string GetJSON() {
+    std::ostringstream os;
+    dmlc::JSONWriter writer(&os);
+    Save(&writer);
+    return os.str();
+  }
+
+ protected:
+  /*!
+   * \brief Add a node to graph.
+   *
+   * \param node A graph node. It is a shared pointer. Some attributes of it
+   *        will be added, i.e. shape and type. These attributes are attached to
+   *        the JSON graph in the end.
+   * \param expr The relay expression.
+   * \return A list of graph entry nodes. It the relay expr is a tuple type, we
+   *         will flatten it.
+   */
+  std::vector<JSONGraphNodeEntry> AddNode(JSONGraphObjectPtr node, const Expr& expr) {
+    auto checked_type = expr->checked_type();
+    auto node_id = nodes_.size();
+    nodes_.push_back(node);
+    std::vector<JSONGraphNodeEntry> ret;
+    ShapeVector shape;
+    TypeVector dtype;
+    // Flatten tuple node.
+    if (const auto* tuple_type = checked_type.as<TupleTypeNode>()) {
+      for (size_t i = 0; i < tuple_type->fields.size(); ++i) {
+        const auto* tensor_type = tuple_type->fields[i].as<TensorTypeNode>();
+        CHECK(tensor_type) << "Expect TensorType, but received: ."
+                           << tuple_type->fields[i]->GetTypeKey();
+        ret.push_back(JSONGraphNodeEntry(node_id, i));
+        shape.emplace_back(GetIntShape(tensor_type->shape));
+        dtype.emplace_back(DType2String(tensor_type->dtype));
+      }
+      node->SetNumOutput(tuple_type->fields.size());
+    } else {
+      const auto* tensor_type = checked_type.as<TensorTypeNode>();
+      CHECK(tensor_type) << "Expect TensorType, but received: " << checked_type->GetTypeKey();
+      shape.emplace_back(GetIntShape(tensor_type->shape));
+      dtype.emplace_back(DType2String(tensor_type->dtype));
+      ret.push_back(JSONGraphNodeEntry(node_id, 0));
+    }
+    std::vector<dmlc::any> shape_attrs;
+    shape_attrs.emplace_back(shape);
+    node->SetAttr("shape", shape_attrs);
+
+    std::vector<dmlc::any> type_attrs;
+    type_attrs.emplace_back(dtype);
+    node->SetAttr("dtype", type_attrs);
+    return ret;
+  }
+
+  void SetCallNodeAttribute(JSONGraphObjectPtr node, const CallNode* cn) {
+    if (cn->op.as<OpNode>()) {
+      OpAttrExtractor extractor(node);
+      const Object* call_attr = cn->attrs.get();
+      extractor.Extract(const_cast<Object*>(call_attr));
+    } else if (const auto* fn = cn->op.as<FunctionNode>()) {
+      auto pattern = fn->GetAttr<String>(attr::kPartitionedFromPattern);
+      CHECK(pattern.defined());
+      std::vector<std::string> values;
+      values.push_back(pattern.value().operator std::string());

Review comment:
       remove `operator std::string()`




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

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



[GitHub] [incubator-tvm] masahi commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
masahi commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-656630522


   Thanks @comaniac @zhiics @mbaret @lhutton1 


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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/dnnl/dnnl_json_runtime.cc
##########
@@ -0,0 +1,455 @@
+/*
+ * 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/dnnl/dnnl_json_runtime.cc
+ * \brief A simple JSON runtime for DNNL.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+#include "dnnl.hpp"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime;
+using namespace tvm::runtime::json;
+
+class DNNLJSONRuntime : public JSONRuntimeBase {
+  using tag = dnnl::memory::format_tag;
+  using dt = dnnl::memory::data_type;
+
+ public:
+  DNNLJSONRuntime(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  const char* type_key() const { return "dnnl_json"; }
+
+  void Init(const Array<NDArray>& consts) override {
+    BuildEngine();
+
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+
+    // Setup constants entries for weights.
+    SetupConstants(consts);
+  }
+
+  void Run() override {
+    // Fill in the input buffers.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto eid = EntryID(input_nodes_[i], 0);
+      // TODO(@comaniac): Support other data lengths.
+      size_t offset_in_bytes = entry_out_mem_[eid].second * 4;
+      size_t buffer_size = GetDataSize(*data_entry_[eid]);
+      write_to_dnnl_memory(data_entry_[eid]->data, entry_out_mem_[eid].first, buffer_size,
+                           offset_in_bytes);
+    }
+
+    // Invoke the engine through intepreting the stream.
+    for (size_t i = 0; i < net_.size(); ++i) {
+      net_.at(i).execute(stream_, net_args_.at(i));
+    }
+    stream_.wait();
+
+    // Read output buffers.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      auto eid = EntryID(outputs_[i]);
+      size_t offset_in_bytes = entry_out_mem_[eid].second * 4;
+      size_t buffer_size = GetDataSize(*data_entry_[eid]);
+      read_from_dnnl_memory(data_entry_[eid]->data, entry_out_mem_[eid].first, buffer_size,
+                            offset_in_bytes);
+    }
+  }
+
+ private:
+  // Build up the engine based on the input graph.
+  void BuildEngine() {
+    engine_ = dnnl::engine(dnnl::engine::kind::cpu, 0);
+    stream_ = dnnl::stream(engine_);
+
+    // Build subgraph engine.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "kernel") {
+        CHECK_EQ(node.GetOpType(), "kernel");
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name) {
+          Conv2d(nid);
+        } else if ("dnnl.conv2d_relu" == op_name) {
+          Conv2d(nid, true, false);
+        } else if ("dnnl.conv2d_bias_relu" == op_name) {
+          Conv2d(nid, true, true);
+        } else if ("nn.dense" == op_name) {
+          Dense(nid);
+        } else if ("nn.batch_norm" == op_name) {
+          BatchNorm(nid);
+        } else if ("nn.relu" == op_name) {
+          Relu(nid);
+        } else if ("add" == op_name) {
+          Add(nid);
+        } else {
+          LOG(FATAL) << "Unsupported op: " << op_name;
+        }
+      }
+    }
+  }
+
+  // Bind a JSON graph node entry to a DNNL memory.
+  dnnl::memory BindDNNLMemory(const JSONGraphNodeEntry& entry, dnnl::memory::desc mem_desc,
+                              size_t offset = 0) {
+    auto eid = EntryID(entry);
+    if (entry_out_mem_.count(eid) == 0) {
+      return BindDNNLMemory(entry, dnnl::memory(mem_desc, engine_), offset);
+    }
+    return entry_out_mem_[eid].first;
+  }
+
+  // Bind a JSON graph node entry to a given DNNL memory.
+  dnnl::memory BindDNNLMemory(const JSONGraphNodeEntry& entry, dnnl::memory mem,
+                              size_t offset = 0) {
+    auto eid = EntryID(entry);
+    // Since the DNNL memory has been created before calling this function, we assume the entry
+    // has not yet been bind to the other DNNL memory; otherwise it may have memory leak.

Review comment:
       bound




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

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



[GitHub] [incubator-tvm] masahi merged pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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


   


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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
comaniac commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-652645424


   We could wait for CI. It should have been updated and included DNNL library already (#5936 ).


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-649147644


   yeah, we should. And we should remove the json_runtime_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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       Is this what you are looking for or you are referring to somethig else?
   https://github.com/apache/incubator-tvm/pull/5919/files/611411df6ec868a34295a240e080c3339b760de1#diff-e7204ab48c8a7f5b4acbd341838634e3R323




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/dnnl/dnnl_json_runtime.cc
##########
@@ -0,0 +1,456 @@
+/*
+ * 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/dnnl/dnnl_json_runtime.cc
+ * \brief A simple JSON runtime for DNNL.
+ */
+
+#include <tvm/runtime/ndarray.h>
+#include <tvm/runtime/registry.h>
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "../json/json_node.h"
+#include "../json/json_runtime.h"
+#include "dnnl.hpp"
+
+namespace tvm {
+namespace runtime {
+namespace contrib {
+
+using namespace tvm::runtime;
+using namespace tvm::runtime::json;
+
+class DNNLJSONRuntime : public JSONRuntimeBase {
+  using tag = dnnl::memory::format_tag;
+  using dt = dnnl::memory::data_type;
+
+ public:
+  DNNLJSONRuntime(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : JSONRuntimeBase(symbol_name, graph_json, const_names) {}
+
+  const char* type_key() const { return "dnnl_json"; }
+
+  void Init(const Array<NDArray>& consts) override {
+    BuildEngine();
+
+    CHECK_EQ(consts.size(), const_idx_.size())
+        << "The number of input constants must match the number of required.";
+
+    // Setup constants entries for weights.
+    SetupConstants(consts);
+  }
+
+  void Run() override {
+    // Fill in the input buffers.
+    for (size_t i = 0; i < input_nodes_.size(); ++i) {
+      auto eid = EntryID(input_nodes_[i], 0);
+      // TODO(@comaniac): Support other data lengths.
+      size_t offset_in_bytes = entry_out_mem_[eid].second * 4;
+      size_t buffer_size = GetDataSize(*data_entry_[eid]);
+      write_to_dnnl_memory(data_entry_[eid]->data, entry_out_mem_[eid].first, buffer_size,
+                           offset_in_bytes);
+    }
+
+    // Invoke the engine through intepreting the stream.
+    for (size_t i = 0; i < net_.size(); ++i) {
+      net_.at(i).execute(stream_, net_args_.at(i));
+    }
+    stream_.wait();
+
+    // Read output buffers.
+    for (size_t i = 0; i < outputs_.size(); ++i) {
+      auto eid = EntryID(outputs_[i]);
+      size_t offset_in_bytes = entry_out_mem_[eid].second * 4;
+      size_t buffer_size = GetDataSize(*data_entry_[eid]);
+      read_from_dnnl_memory(data_entry_[eid]->data, entry_out_mem_[eid].first, buffer_size,
+                            offset_in_bytes);
+    }
+  }
+
+ private:
+  // Build up the engine based on the input graph.
+  void BuildEngine() {
+    engine_ = dnnl::engine(dnnl::engine::kind::cpu, 0);
+    stream_ = dnnl::stream(engine_);
+
+    // Build subgraph engine.
+    for (size_t nid = 0; nid < nodes_.size(); ++nid) {
+      const auto& node = nodes_[nid];
+      if (node.GetOpType() == "kernel") {
+        CHECK_EQ(node.GetOpType(), "kernel");
+        auto op_name = node.GetOpName();
+        if ("nn.conv2d" == op_name) {
+          Conv2d(nid);
+        } else if ("dnnl.conv2d_relu" == op_name) {
+          Conv2d(nid, true, false);
+        } else if ("dnnl.conv2d_bias_relu" == op_name) {
+          Conv2d(nid, true, true);
+        } else if ("nn.dense" == op_name) {
+          Dense(nid);
+        } else if ("nn.batch_norm" == op_name) {
+          BatchNorm(nid);
+        } else if ("nn.relu" == op_name) {
+          Relu(nid);
+        } else if ("add" == op_name) {
+          Add(nid);
+        } else {
+          LOG(FATAL) << "Unsupported op: " << op_name;
+        }
+      }
+    }
+  }
+
+  // Bind a JSON graph node entry to a DNNL memory.
+  dnnl::memory BindDNNLMemory(const JSONGraphNodeEntry& entry, dnnl::memory::desc mem_desc,
+                              size_t offset = 0) {
+    auto eid = EntryID(entry);
+    if (entry_out_mem_.count(eid) == 0) {
+      return BindDNNLMemory(entry, dnnl::memory(mem_desc, engine_), offset);
+    }
+    return entry_out_mem_[eid].first;
+  }
+
+  // Bind a JSON graph node entry to a given DNNL memory.
+  dnnl::memory BindDNNLMemory(const JSONGraphNodeEntry& entry, dnnl::memory mem,
+                              size_t offset = 0) {
+    auto eid = EntryID(entry);
+    // Since the DNNL memory has been created before calling this function, we assume the entry
+    // has not yet been bind to the other DNNL memory; otherwise it may have memory leak.
+    CHECK_EQ(entry_out_mem_.count(eid), 0);
+
+    // TODO(@comanic): Support other data types (i.e., int8).
+    auto data_node = nodes_[entry.id_];
+    auto dltype = data_node.GetOpDataType()[entry.index_];
+    CHECK_EQ(dltype.bits, 32);
+
+    entry_out_mem_[eid] = {mem, offset};
+    return entry_out_mem_[eid].first;
+  }
+
+  void Conv2d(const size_t& nid, const bool has_relu = false, const bool has_bias = false) {
+    auto node = nodes_[nid];
+
+    // Setup attributes.
+    auto data_entry = node.GetInputs()[0];
+    auto weight_entry = node.GetInputs()[1];
+    dnnl::memory::dims input_shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    dnnl::memory::dims weight_shape = nodes_[weight_entry.id_].GetOpShape()[weight_entry.index_];
+    std::vector<std::string> str_strides = node.GetAttr<std::vector<std::string>>("strides");
+    std::vector<std::string> str_padding = node.GetAttr<std::vector<std::string>>("padding");
+    dnnl::memory::dim groups = std::stoi(node.GetAttr<std::vector<std::string>>("groups")[0]);
+
+    dnnl::memory::dim N = input_shape[0],       // batch size
+        IC = input_shape[1],                    // input channels
+        IH = input_shape[2],                    // input height
+        IW = input_shape[2],                    // input width
+        OC = weight_shape[0],                   // output channels
+        KH = weight_shape[2],                   // weight height
+        KW = weight_shape[3],                   // weight width
+        PH_L = std::stoi(str_padding[1]),       // height padding: left
+        PH_R = std::stoi(str_padding[3]),       // height padding: right
+        PW_L = std::stoi(str_padding[0]),       // width padding: left
+        PW_R = std::stoi(str_padding[2]),       // width padding: right
+        SH = std::stoi(str_strides[0]),         // height-wise stride
+        SW = std::stoi(str_strides[0]),         // weight-wise stride
+        OH = (IH - KH + PH_L + PH_R) / SH + 1,  // output height
+        OW = (IW - KW + PW_L + PW_R) / SW + 1;  // output width
+
+    // Memory shapes.
+    dnnl::memory::dims src_dims = {N, IC, IH, IW};
+    dnnl::memory::dims weights_dims = {OC, IC, KH, KW};
+    if (groups > 1) {
+      weights_dims = {groups, 1, IC / groups, KH, KW};
+    }
+    dnnl::memory::dims bias_dims = {OC};
+    dnnl::memory::dims dst_dims = {N, OC, OH, OW};
+    dnnl::memory::dims strides_dims = {SH, SW};
+    dnnl::memory::dims padding_dims_l = {PH_L, PW_L};
+    dnnl::memory::dims padding_dims_r = {PH_R, PW_R};
+
+    // Memory descriptions.
+    auto conv_src_md = dnnl::memory::desc(src_dims, dt::f32, tag::any);
+    auto conv_weights_md = dnnl::memory::desc(weights_dims, dt::f32, tag::any);
+    auto conv_bias_md = dnnl::memory::desc(bias_dims, dt::f32, tag::any);
+    auto conv_dst_md = dnnl::memory::desc(dst_dims, dt::f32, tag::nchw);
+
+    // Covn2d description.
+    auto conv_desc = dnnl::convolution_forward::desc(
+        dnnl::prop_kind::forward_inference, dnnl::algorithm::convolution_direct, conv_src_md,
+        conv_weights_md, conv_bias_md, conv_dst_md, strides_dims, padding_dims_l, padding_dims_r);
+
+    // Enable ReLU
+    dnnl::primitive_attr attr;
+    if (has_relu) {
+      dnnl::post_ops ops;
+      ops.append_eltwise(1.f, dnnl::algorithm::eltwise_relu, 0.f, 0.f);
+      attr.set_post_ops(ops);
+    }
+
+    auto conv2d_prim_desc = dnnl::convolution_forward::primitive_desc(conv_desc, attr, engine_);
+
+    // Push to the network.
+    auto conv = dnnl::convolution_forward(conv2d_prim_desc);
+    net_.push_back(conv);
+
+    // Data memory.
+    CHECK_EQ(node.GetAttr<std::vector<std::string>>("data_layout")[0], "NCHW");
+    auto conv2d_src_memory = BindDNNLMemory(data_entry, {src_dims, dt::f32, tag::nchw});
+
+    // Weight memory.
+    CHECK_EQ(node.GetAttr<std::vector<std::string>>("kernel_layout")[0], "OIHW");
+    auto conv2d_weights_memory = BindDNNLMemory(
+        weight_entry, {weights_dims, dt::f32, (groups > 1) ? tag::goihw : tag::oihw});
+
+    // Bias memory.
+    auto conv2d_bias_memory = dnnl::memory({bias_dims, dt::f32, tag::x}, engine_);
+    if (has_bias) {
+      auto bias_entry = node.GetInputs()[2];
+      BindDNNLMemory(bias_entry, conv2d_bias_memory);
+    } else {
+      float bias[OC] = {0};
+      write_to_dnnl_memory(bias, conv2d_bias_memory, OC * sizeof(float));
+    }
+
+    // Output memory.
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto conv2d_dst_memory = BindDNNLMemory(out_entry, conv2d_prim_desc.dst_desc());
+
+    // Bind memory buffers.
+    net_args_.push_back({{DNNL_ARG_SRC, conv2d_src_memory},
+                         {DNNL_ARG_WEIGHTS, conv2d_weights_memory},
+                         {DNNL_ARG_BIAS, conv2d_bias_memory},
+                         {DNNL_ARG_DST, conv2d_dst_memory}});
+  }
+
+  void Dense(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    // Setup attributes.
+    auto data_entry = node.GetInputs()[0];
+    auto weight_entry = node.GetInputs()[1];
+    dnnl::memory::dims input_shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    dnnl::memory::dims weight_shape = nodes_[weight_entry.id_].GetOpShape()[weight_entry.index_];
+
+    dnnl::memory::dim B = input_shape[0],  // batch size
+        IC = input_shape[1],               // input channels
+        OC = weight_shape[0];              // output channels
+
+    // Memory shapes.
+    dnnl::memory::dims data_dims = {B, IC};
+    dnnl::memory::dims weight_dims = {OC, IC};
+    dnnl::memory::dims bias_dims = {OC};
+    dnnl::memory::dims out_dims = {B, OC};
+
+    // Memory descriptions.
+    auto data_md = dnnl::memory::desc({data_dims, dt::f32, tag::nc});
+    auto weight_md = dnnl::memory::desc({weight_dims, dt::f32, tag::nc});
+    auto bias_md = dnnl::memory::desc({bias_dims, dt::f32, tag::x});
+    auto dst_md = dnnl::memory::desc({out_dims, dt::f32, tag::nc});
+
+    // Dense description.
+    auto dense_desc = dnnl::inner_product_forward::desc(dnnl::prop_kind::forward_inference, data_md,
+                                                        weight_md, bias_md, dst_md);
+    auto dense_prim_desc = dnnl::inner_product_forward::primitive_desc(dense_desc, engine_);
+
+    auto dense = dnnl::inner_product_forward(dense_prim_desc);
+    net_.push_back(dense);
+
+    // Memories.
+    auto data_memory = BindDNNLMemory(data_entry, data_md);
+    auto weight_memory = BindDNNLMemory(weight_entry, weight_md);
+    auto bias_memory = dnnl::memory(bias_md, engine_);
+    float bias[OC] = {0};
+    write_to_dnnl_memory(bias, bias_memory, OC * sizeof(float));
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto dst_memory = BindDNNLMemory(out_entry, dense_prim_desc.dst_desc());
+
+    net_args_.push_back({{DNNL_ARG_SRC, data_memory},
+                         {DNNL_ARG_WEIGHTS, weight_memory},
+                         {DNNL_ARG_BIAS, bias_memory},
+                         {DNNL_ARG_DST, dst_memory}});
+  }
+
+  void BatchNorm(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    auto data_entry = node.GetInputs()[0];
+    auto gamma_entry = node.GetInputs()[1];
+    auto beta_entry = node.GetInputs()[2];
+    auto mean_entry = node.GetInputs()[3];
+    auto variance_entry = node.GetInputs()[4];
+    dnnl::memory::dims data_shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    dnnl::memory::dim IC = data_shape[1];
+    float epsilon = std::stof(node.GetAttr<std::vector<std::string>>("epsilon")[0]);
+
+    // Memory description.
+    dnnl::memory::desc data_md = GenDNNLMemDescByShape(data_shape, dt::f32);
+
+    // BN description.
+    auto bn_desc = dnnl::batch_normalization_forward::desc(
+        dnnl::prop_kind::forward_inference, data_md, epsilon,
+        dnnl::normalization_flags::use_global_stats | dnnl::normalization_flags::use_scale_shift);
+    auto bn_prim_desc = dnnl::batch_normalization_forward::primitive_desc(bn_desc, engine_);
+    auto bn = dnnl::batch_normalization_forward(bn_prim_desc);
+    net_.push_back(bn);
+
+    // Memories.
+    auto data_memory = BindDNNLMemory(data_entry, data_md);
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto out_memory = BindDNNLMemory(out_entry, data_md);
+    auto mean_memory = BindDNNLMemory(mean_entry, bn_prim_desc.mean_desc());
+    auto variance_memory = BindDNNLMemory(variance_entry, bn_prim_desc.variance_desc());
+
+    // In DNNL, weight is composed of gamma+beta, so we point them to the same DNNL memory but
+    // assign an offset to beta data for runtime serialization.
+    auto weight_memory = BindDNNLMemory(gamma_entry, bn_prim_desc.weights_desc(), 0);
+    BindDNNLMemory(beta_entry, weight_memory, IC);
+
+    net_args_.push_back({{DNNL_ARG_SRC, data_memory},
+                         {DNNL_ARG_DST, out_memory},
+                         {DNNL_ARG_SCALE_SHIFT, weight_memory},
+                         {DNNL_ARG_MEAN, mean_memory},
+                         {DNNL_ARG_VARIANCE, variance_memory}});
+  }
+
+  void Relu(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    auto data_entry = node.GetInputs()[0];
+    dnnl::memory::dims shape = nodes_[data_entry.id_].GetOpShape()[data_entry.index_];
+    auto data_md = dnnl::memory::desc{{shape}, dt::f32, tag::abcd};
+
+    auto relu_desc = dnnl::eltwise_forward::desc(dnnl::prop_kind::forward_inference,
+                                                 dnnl::algorithm::eltwise_relu, data_md, 0);
+    auto relu_prim_desc = dnnl::eltwise_forward::primitive_desc(relu_desc, engine_);
+    CHECK(data_md == relu_prim_desc.dst_desc());
+
+    auto relu = dnnl::eltwise_forward(relu_prim_desc);
+    net_.push_back(relu);
+
+    auto data_memory = BindDNNLMemory(data_entry, data_md);
+    auto out_md = dnnl::memory::desc(shape, dt::f32, tag::abcd);
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto out_memory = BindDNNLMemory(out_entry, out_md);
+
+    net_args_.push_back({{DNNL_ARG_SRC, data_memory}, {DNNL_ARG_DST, out_memory}});
+  }
+
+  void Add(const size_t& nid) {
+    auto node = nodes_[nid];
+
+    // Memory and compute description.
+    std::vector<dnnl::memory::dims> data_dims;
+    std::vector<dnnl::memory::desc> data_mds;
+    std::vector<dnnl::memory> data_memories;
+
+    CHECK_EQ(node.GetInputs().size(), 2U);
+    for (auto entry : node.GetInputs()) {
+      auto data_shape = nodes_[entry.id_].GetOpShape()[entry.index_];
+      dnnl::memory::desc data_md = GenDNNLMemDescByShape(data_shape, dt::f32);
+
+      data_dims.push_back(data_shape);
+      data_mds.push_back(data_md);
+      data_memories.push_back(BindDNNLMemory(entry, data_md));
+    }
+    CHECK(data_dims[0] == data_dims[1]);
+    auto out_md = data_mds[0];
+    JSONGraphNodeEntry out_entry(nid, 0);
+    auto out_memory = BindDNNLMemory(out_entry, out_md);
+
+    auto add_desc =
+        dnnl::binary::desc(dnnl::algorithm::binary_add, data_mds[0], data_mds[1], out_md);
+    auto add_prim_desc = dnnl::binary::primitive_desc(add_desc, engine_);
+    auto add = dnnl::binary(add_prim_desc);
+    net_.push_back(add);
+
+    net_args_.push_back({{DNNL_ARG_SRC_0, data_memories[0]},
+                         {DNNL_ARG_SRC_1, data_memories[1]},
+                         {DNNL_ARG_DST, out_memory}});
+  }
+
+  // Read from DNNL memory (+offset) and write to the handle.
+  inline void read_from_dnnl_memory(void* handle, const dnnl::memory& mem, size_t size,
+                                    size_t offset = 0) {
+    uint8_t* src = static_cast<uint8_t*>(mem.get_data_handle());
+    std::copy(src + offset, src + offset + size, static_cast<uint8_t*>(handle));
+  }
+
+  // Read from the handle and write to DNNL memory (+offset).
+  inline void write_to_dnnl_memory(void* handle, const dnnl::memory& mem, size_t size,
+                                   size_t offset = 0) {
+    uint8_t* dst = static_cast<uint8_t*>(mem.get_data_handle());
+    std::copy(reinterpret_cast<uint8_t*>(handle), reinterpret_cast<uint8_t*>(handle) + size,
+              dst + offset);
+  }
+
+  // Generate DNNL memory description and infer the data layout by the given shape.
+  inline dnnl::memory::desc GenDNNLMemDescByShape(const dnnl::memory::dims& shape, dt dtype) {
+    dnnl::memory::desc data_md;
+    switch (shape.size()) {
+      case 2:
+        data_md = dnnl::memory::desc({shape, dtype, tag::ab});
+        break;
+      case 3:
+        data_md = dnnl::memory::desc({shape, dtype, tag::abc});
+        break;
+      case 4:
+        data_md = dnnl::memory::desc({shape, dtype, tag::abcd});
+        break;
+      case 5:
+        data_md = dnnl::memory::desc({shape, dtype, tag::abcde});
+        break;
+      default:
+        LOG(FATAL) << "Unsupported data shape dimension: " << shape.size();
+        break;
+    }
+    return data_md;
+  }
+
+  /* The dnnl engine. */
+  dnnl::engine engine_;
+  /* The dnnl stream. */
+  dnnl::stream stream_;
+  /* The network layers that are represented in dnnl primitives. */
+  std::vector<dnnl::primitive> net_;
+  /* The memory that is consumed by arguments. */
+  std::vector<std::unordered_map<int, dnnl::memory>> net_args_;
+  /* The entry ID to its corresponding output memory. */
+  std::unordered_map<uint32_t, std::pair<dnnl::memory, size_t>> entry_out_mem_;
+};
+
+runtime::Module DNNLJSONRuntimeCreate(String symbol_name, String graph_json,
+                                      const Array<String>& const_names) {
+  auto n = make_object<DNNLJSONRuntime>(symbol_name.operator std::string(),

Review comment:
       There are many occurences of `.operator std::string()` in the PR, but since `String` to `std::string` implicit conversion is defined, I think these explicit conversion can be removed.




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());

Review comment:
       remove `operator std::string()`




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       The error seems to be about `std::vector`, so I don't think `Handler<std::unordered_map<std::string, dmlc::any>>` is related?




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

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



[GitHub] [incubator-tvm] mbaret commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_runtime.h
##########
@@ -0,0 +1,267 @@
+/*
+ * 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/json/json_runtime.h
+ * \brief Utilities for json runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_RUNTIME_H_
+
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/ndarray.h>
+
+#include <cstddef>
+#include <string>
+#include <tuple>
+#include <type_traits>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "json_node.h"
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+/*!
+ * \brief A json runtime that executes the serialized JSON format. This runtime
+ * can be extended by user defined runtime for execution.
+ */
+class JSONRuntimeBase : public ModuleNode {
+ public:
+  JSONRuntimeBase(const std::string& symbol_name, const std::string& graph_json,
+                  const Array<String> const_names)
+      : symbol_name_(symbol_name), graph_json_(graph_json), const_names_(const_names) {
+    LoadGraph(graph_json_);
+  }
+
+  const char* type_key() const { return "json"; }
+
+  /*! \brief Initialize a specific json runtime. */
+  virtual void Init(const Array<NDArray>& consts) = 0;
+
+  /*! \brief Invoke the execution engine to inteprete a specific json runtime. */
+  virtual void Run() = 0;
+
+  /*!
+   * \brief Get a packed function.
+   * \param name The name/symbol of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The packed function.
+   */
+  virtual PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_symbol") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->symbol_name_; });
+    } else if (name == "get_const_vars") {
+      return PackedFunc(
+          [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->const_names_; });
+    } else if (this->symbol_name_ == name) {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK(this->initialized_) << "The module has not been initialized";
+
+        // Bind argument tensors to data entries.
+        this->SetInputOutputBuffers(args);
+        // Execute the subgraph.
+        this->Run();
+      });
+    } else if ("__init_" + this->symbol_name_ == name) {
+      // The function to initialize constant tensors.
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        CHECK_EQ(args.size(), 1U);
+        this->Init(args[0]);
+        this->initialized_ = true;
+        *rv = 0;
+      });
+    } else {
+      return PackedFunc(nullptr);
+    }
+  }
+
+  virtual void SaveToBinary(dmlc::Stream* stream) {
+    // Save the symbol
+    stream->Write(symbol_name_);
+    // Save the graph
+    stream->Write(graph_json_);
+    // Save the required const names
+    std::vector<std::string> consts;
+    for (const auto& it : const_names_) {
+      consts.push_back(it);
+    }
+    stream->Write(consts);
+  }
+
+  template <typename T,
+            typename = typename std::enable_if<std::is_base_of<JSONRuntimeBase, T>::value>::type>
+  static Module LoadFromBinary(void* strm) {
+    dmlc::Stream* stream = static_cast<dmlc::Stream*>(strm);
+    std::string symbol;
+    std::string graph_json;
+    std::vector<std::string> consts;
+    // Load the symbol
+    CHECK(stream->Read(&symbol)) << "Loading symbol name failed";

Review comment:
       OK, good to hear.




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

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



[GitHub] [incubator-tvm] masahi commented on pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

Posted by GitBox <gi...@apache.org>.
masahi commented on pull request #5919:
URL: https://github.com/apache/incubator-tvm/pull/5919#issuecomment-652641725


   Do we want to wait until dnnl is up on the CI? And what about @zhiics's comment below.
   
   >  And we should remove the json_runtime_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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/runtime/contrib/json/json_node.h
##########
@@ -0,0 +1,358 @@
+/*
+ * 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/json/json_node.h
+ * \brief The graph nodes used by JSON runtime.
+ */
+
+#ifndef TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+#define TVM_RUNTIME_CONTRIB_JSON_JSON_NODE_H_
+
+#include <dlpack/dlpack.h>
+#include <dmlc/json.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/container.h>
+
+#include <cstdint>
+#include <cstdio>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+namespace json {
+
+using namespace tvm::runtime;
+using JSONGraphAttrs = std::unordered_map<std::string, dmlc::any>;
+
+/*!
+ * \brief The node entry in the serialized json graph.
+ */
+class JSONGraphNodeEntry {
+ public:
+  // Constructors.
+  JSONGraphNodeEntry() = default;
+  JSONGraphNodeEntry(int id, int index, int version = 0)
+      : id_(id), index_(index), version_(version) {}
+
+  /*!
+   * \brief Serialize a node entry.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) const {
+    writer->BeginArray();
+    writer->WriteArrayItem(id_);
+    writer->WriteArrayItem(index_);
+    writer->WriteArrayItem(version_);
+    writer->EndArray();
+  }
+
+  /*!
+   * \brief Deserialize the json string into a node entry.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginArray();
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&id_);
+    CHECK(reader->NextArrayItem()) << "invalid json format";
+    reader->Read(&index_);
+    if (reader->NextArrayItem()) {
+      reader->Read(&version_);
+      CHECK(!reader->NextArrayItem()) << "invalid json format";
+    } else {
+      version_ = 0;
+    }
+  }
+
+  /*! \brief The json graph node ID. */
+  uint32_t id_;
+  /*! \brief The entry index. */
+  uint32_t index_;
+  uint32_t version_;
+};
+
+/*!
+ * \brief The node of the serialized json graph. It includes an array of
+ * entries.
+ */
+class JSONGraphNode {
+ public:
+  // Constructors.
+  JSONGraphNode() = default;
+  JSONGraphNode(const std::string& name, const std::string& op_type,
+                const std::vector<JSONGraphNodeEntry>& inputs = {}, size_t num_outputs = 1) {
+    name_ = name;
+    op_type_ = op_type;
+    num_inputs_ = inputs.size();
+    inputs_ = inputs;
+    num_outputs_ = num_outputs;
+  }
+
+  /*!
+   * \brief Serialize a node so that it can be saved to disk.
+   * \param writer The json writer.
+   */
+  void Save(dmlc::JSONWriter* writer) {
+    writer->BeginObject();
+    writer->WriteObjectKeyValue("op", op_type_);
+    writer->WriteObjectKeyValue("name", name_);
+    if (!inputs_.empty()) {
+      SetAttr("num_inputs", std::to_string(inputs_.size()));
+      SetAttr("num_outputs", std::to_string(num_outputs_));
+      writer->WriteObjectKeyValue("inputs", this->inputs_);
+    }
+    if (!attrs_.empty()) {
+      writer->WriteObjectKeyValue("attrs", attrs_);
+    }
+    writer->EndObject();
+  }
+
+  /*!
+   * \brief Load the attribute of a node in the json string.
+   * \param reader The json reader.
+   */
+  void LoadAttrs(dmlc::JSONReader* reader) {
+    std::string key, value;
+    reader->BeginObject();
+    while (reader->NextObjectItem(&key)) {
+      if (key == "num_inputs") {
+        reader->Read(&value);
+        num_inputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "num_outputs") {
+        reader->Read(&value);
+        num_outputs_ = strtoul(value.c_str(), nullptr, 10);
+      } else if (key == "dtype") {
+        std::vector<std::string> tmp;
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&tmp);
+        CHECK(!reader->NextArrayItem());
+        for (const auto& it : tmp) {
+          dtype_.push_back(tvm::runtime::String2DLDataType(it));
+        }
+      } else if (key == "shape") {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        reader->Read(&shape_);
+        CHECK(!reader->NextArrayItem());
+      } else {
+        reader->BeginArray();
+        CHECK(reader->NextArrayItem());
+        std::vector<std::string> tmp;
+        reader->Read(&tmp);
+        attrs_[key] = tmp;
+        CHECK(!reader->NextArrayItem());
+      }
+    }
+    CHECK_EQ(shape_.size(), dtype_.size());
+  }
+
+  /*!
+   * \brief Load a node in the json string.
+   * \param reader The json reader.
+   */
+  void Load(dmlc::JSONReader* reader) {
+    reader->BeginObject();
+    std::string key;
+    while (reader->NextObjectItem(&key)) {
+      if (key == "op") {
+        reader->Read(&op_type_);
+      } else if (key == "name") {
+        reader->Read(&name_);
+      } else if (key == "inputs") {
+        reader->Read(&inputs_);
+      } else if (key == "attr" || key == "attrs") {
+        this->LoadAttrs(reader);
+      } else {
+        LOG(FATAL) << "Unknown key: " << key;
+      }
+    }
+  }
+
+  /*!
+   * \brief Check if a node is a leaf node, i.e. input to the graph.
+   *
+   * \return True if the node has no input, otherwise, false.
+   */
+  bool IsLeaf() const { return inputs_.empty(); }
+
+  /*!
+   * \brief Return the number of outputs of the node.
+   *
+   * \return The number of the output.
+   */
+  uint32_t GetNumOutput() const { return num_outputs_; }
+
+  /*!
+   * \brief Return the input entries.
+   *
+   * \return The input entries.
+   */
+  std::vector<JSONGraphNodeEntry> GetInputs() const { return inputs_; }
+
+  /*!
+   * \brief Return the op type.
+   *
+   * \return The op type.
+   */
+  std::string GetOpType() const { return op_type_; }
+
+  /*!
+   * \brief Return the op name.
+   *
+   * \return The op name.
+   */
+  std::string GetOpName() const { return name_; }
+
+  /*!
+   * \brief Return the op output shapes.
+   *
+   * \return The shapes.
+   */
+  std::vector<std::vector<int64_t>> GetOpShape() const { return shape_; }
+
+  /*!
+   * \brief Return the op types.
+   *
+   * \return The types.
+   */
+  std::vector<DLDataType> GetOpDataType() const { return dtype_; }
+
+  /*!
+   * \brief Set the number of outputs of the node.
+   *
+   * \param num_outputs The number of output.
+   */
+  void SetNumOutput(uint32_t num_outputs) { num_outputs_ = num_outputs; }
+
+  /*!
+   * \brief Get the value of an attribute in the node.
+   *
+   * \tparam T The return type.
+   * \param key The key for lookup.
+   *
+   * \return The value.
+   */
+  template <typename T>
+  T GetAttr(const std::string& key) const {
+    CHECK_GT(attrs_.count(key), 0U) << "Key: " << key << "is not found";
+    return dmlc::get<T>(attrs_.at(key));
+  }
+
+  /*!
+   * \brief Set an attribute for the node.
+   *
+   * \tparam ValueT The type of the value being stored.
+   * \param key The key of the attribute.
+   * \param value The value of the attribute.
+   */
+  template <typename ValueT>
+  void SetAttr(const std::string& key, const ValueT& value) {
+    attrs_[key] = value;
+  }
+
+  virtual ~JSONGraphNode() {}
+
+ private:
+  /*! \brief The number of input. */
+  uint32_t num_inputs_{0};
+  /*! \brief The number of output. */
+  uint32_t num_outputs_{1};
+  /*! \brief The name of the op. It is the symbol that used for runtime lookup. */
+  std::string name_;
+  /*! \brief The operator type, i.e. input is "null". */
+  std::string op_type_;
+  /*! \brief The shape of the node. */
+  std::vector<std::vector<int64_t>> shape_;
+  /*! \brief The type of the node. */
+  std::vector<DLDataType> dtype_;
+  /*! \brief The inputs of the node. */
+  std::vector<JSONGraphNodeEntry> inputs_;
+  /*!
+   * \brief Attribute of the node. For simplicity, we store all attribute as
+   * a list of std::string. It's the developer's resposibility to check the
+   * required attribute of a certain op and convert it into the needed type.
+   *
+   * For example, for conv2d, this map could contain:
+   *  attrs_["strides"] = ["1", "1"]

Review comment:
       ok my local copy was pointing to the commit 2 days ago. Updating to the latest fixed this. 
   
   Somehow the change that added `Handler<std::unordered_map<std::string, dmlc::any>>` did the trick?
   
   




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

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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #5919: [BYOC] JSON Runtime with DNNL End-to-End Flow

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



##########
File path: src/relay/backend/contrib/codegen_json/codegen_json.h
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 relay/backend/contrib/codegen_json.h
+ * \brief Utilities for json codegen and runtime
+ */
+#ifndef TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+#define TVM_RELAY_BACKEND_CONTRIB_CODEGEN_JSON_CODEGEN_JSON_H_
+
+#include <dmlc/any.h>
+#include <dmlc/json.h>
+#include <tvm/node/container.h>
+#include <tvm/node/reflection.h>
+#include <tvm/runtime/container.h>
+#include <tvm/tir/op.h>
+
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "../../../../runtime/contrib/json/json_node.h"
+#include "../../../../runtime/contrib/json/json_runtime.h"
+#include "../../utils.h"
+
+namespace tvm {
+namespace relay {
+namespace backend {
+namespace contrib {
+
+using namespace tvm::runtime::json;
+
+using ShapeVector = std::vector<std::vector<int64_t>>;
+using TypeVector = std::vector<std::string>;
+using JSONGraphObjectPtr = std::shared_ptr<JSONGraphNode>;
+
+/*!
+ * \brief Helper class to extract all attributes of a certain op and save them
+ * into text format.
+ */
+class OpAttrExtractor : public AttrVisitor {
+ public:
+  explicit OpAttrExtractor(JSONGraphObjectPtr node) : node_(node) {}
+
+  template <typename T = double, typename = std::enable_if_t<std::is_floating_point<T>::value>>
+  std::string Fp2String(const T value) {
+    std::ostringstream out;
+    out.precision(std::numeric_limits<T>::max_digits10);
+    out << value;
+    return out.str();
+  }
+
+  void SetNodeAttr(const char* key, const std::vector<std::string>& value) {
+    std::vector<dmlc::any> attr;
+    attr.emplace_back(value);
+    node_->SetAttr(key, attr);
+  }
+
+  void Visit(const char* key, double* value) final { SetNodeAttr(key, {Fp2String(*value)}); }
+
+  void Visit(const char* key, int64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, uint64_t* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, int* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, bool* value) final { SetNodeAttr(key, {std::to_string(*value)}); }
+
+  void Visit(const char* key, std::string* value) final { SetNodeAttr(key, {*value}); }
+
+  void Visit(const char* key, DataType* value) final {
+    if (!value->is_void()) {
+      SetNodeAttr(key, {runtime::DLDataType2String(*value)});
+    } else {
+      SetNodeAttr(key, {""});
+    }
+  }
+
+  void Visit(const char* key, runtime::ObjectRef* value) final {
+    if (const auto* an = (*value).as<ArrayNode>()) {
+      std::vector<std::string> attr;
+      for (size_t i = 0; i < an->size(); ++i) {
+        if (const auto* im = (*an)[i].as<IntImmNode>()) {
+          attr.push_back(std::to_string(im->value));
+        } else if (const auto* fm = (*an)[i].as<FloatImmNode>()) {
+          attr.push_back(Fp2String(fm->value));
+        } else if (const auto* str = (*an)[i].as<StringObj>()) {
+          String s = GetRef<String>(str);
+          attr.push_back(s.operator std::string());
+        } else {
+          LOG(FATAL) << "Not supported type: " << (*an)[i]->GetTypeKey();
+        }
+      }
+      SetNodeAttr(key, attr);
+    } else if (!(*value).defined()) {  // Skip NullValue
+      SetNodeAttr(key, std::vector<std::string>{""});
+    } else if (const auto* im = (*value).as<IntImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{std::to_string(im->value)});
+    } else if (const auto* fm = (*value).as<FloatImmNode>()) {
+      SetNodeAttr(key, std::vector<std::string>{Fp2String(fm->value)});
+    } else if (const auto* str = (*value).as<StringObj>()) {
+      String s = GetRef<String>(str);
+      SetNodeAttr(key, std::vector<std::string>{s.operator std::string()});
+    } else {
+      LOG(FATAL) << "Not yet supported type: " << (*value)->GetTypeKey() << ": " << *value;
+    }
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    LOG(FATAL) << "NDArray is not allowed in op attribute";
+  }
+
+  void Visit(const char* key, void** value) final {
+    LOG(FATAL) << "void pointer is not allowed in op attribute";
+  }
+
+  void Extract(Object* node) {
+    if (node) {
+      reflection_->VisitAttrs(node, this);
+    }
+  }
+
+ private:
+  JSONGraphObjectPtr node_;
+  ReflectionVTable* reflection_ = ReflectionVTable::Global();

Review comment:
       What is the purpose of `reflection_` and `Extract()` method above?




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