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/01/08 12:15:28 UTC

[GitHub] [incubator-tvm] FrozenGene opened a new pull request #4657: [CodeGen] Generate blob use LLVM directly

FrozenGene opened a new pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657
 
 
   This is one prior work of[ Module based Model Runtime Interface RFC](https://discuss.tvm.ai/t/discuss-module-based-model-runtime-interface/5025). In this RFC, we want to serialize params (weight) into shared library directly. 
   
   However, previous way of serializing the blob is to use `PackImportToC`, i.e. we will serialize the blob into char array `__tvm_dev_blob`, which stored the serialized result (0x...0x...), then we write this into `dev.cc` and pass to the compiler compiling. If we don't serialize params, it is ok. However, when we serialize params, the `__tvm_dev_blob` and `dev.cc` will become very large (resnet18 workload of `dev.cc` will be 380M, without serializing weight is 2.6M). The compiling time will increase very much (**Even 10x slower**). 
   
   According to the investigation, the bottleneck is the parsing time (see [GCC time report](https://discuss.tvm.ai/t/discuss-module-based-model-runtime-interface/5025/5?u=frozengene)).
   
   So we decide to avoid the C++ compiler (i.e. avoid the parsing time) and generate LLVM module directly. After testing, even we we serialize the params into shared library, the compiling time of generating blob is very fast. The compiling time of resnet18 on one cloud machine I tested before could **boost from 128.46s to 13.28s**. see[ previous test](https://discuss.tvm.ai/t/discuss-module-based-model-runtime-interface/5025/17?u=frozengene)
   
   After this work, I find it also have some a little improvement of compiling time compared previous way testing on my local machine (Intel i7-7700 CPU @ 3.60GHz). 
   
   Workload is restnet18 on CUDA. Run 10 times and get the average
   
   | PackImportToC | PackImportToLLVM |
   | ---------------------| ----------------|
   | 8.586s             | 8.082s      |
   
   @tqchen @zhiics @yzhliu @icemelon9 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365065438
 
 

 ##########
 File path: src/codegen/llvm/llvm_module.cc
 ##########
 @@ -62,6 +63,11 @@ class LLVMModuleNode final : public runtime::ModuleNode {
       return PackedFunc([flag](TVMArgs args, TVMRetValue *rv) {
           * rv = flag;
         });
+    } else if (name == "get_target_triple") {
 
 Review comment:
   Will do

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364361636
 
 

 ##########
 File path: include/tvm/codegen.h
 ##########
 @@ -59,6 +59,21 @@ runtime::Module Build(const Array<LoweredFunc>& funcs,
  * \return cstr The C string representation of the file.
  */
 std::string PackImportsToC(const runtime::Module& m, bool system_lib);
+
+/*!
+ * \brief Pack imported device library to a LLVM module.
+ *  Compile the LLVM module and link with the host library
+ *  will allow the DSO loader to automatically discover and import
+ *  the dependency from the shared library.
+ *
+ * \param m The host module with the imports.
+ * \param system_lib Whether expose as system library.
+ * \param target LLVM target
+ * \return runtime::Module The generated LLVM module.
+ */
+runtime::Module PackImportsToLLVM(const runtime::Module& m,
+                                  bool system_lib,
+                                  const std::string& target);
 
 Review comment:
   use `Target` here looks better than 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365066395
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   I specificlaly mean add support here https://github.com/apache/incubator-tvm/blob/master/python/tvm/contrib/cc.py#L107
   
   By allowing passing in an optional get_target_triple function

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365062069
 
 

 ##########
 File path: src/codegen/llvm/codegen_blob.cc
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * 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 codegen_blob.cc
+ */
+#ifdef TVM_LLVM_VERSION
+#include <tvm/runtime/module.h>
+#include <cstring>
+#include "codegen_blob.h"
+
+namespace tvm {
+namespace codegen {
+
+std::pair<std::unique_ptr<llvm::Module>,
+          std::shared_ptr<llvm::LLVMContext>> CodeGenBlob(const std::string& data,
+                                                          bool system_lib,
+                                                          const std::string& target) {
+  InitializeLLVM();
+  auto tm = GetLLVMTargetMachine(std::string("-target ") + target);
+  auto target_triple = tm->getTargetTriple();
+  auto ctx = std::make_shared<llvm::LLVMContext>();
+  std::string module_name = "devc";
+  std::unique_ptr<llvm::Module> module(new llvm::Module(module_name, *ctx));
+  module->setTargetTriple(target_triple.str());
+  module->setDataLayout(tm->createDataLayout());
+  auto* blob_value = llvm::ConstantDataArray::getString(*ctx, data, false);
+  auto* tvm_dev_mblob = new llvm::GlobalVariable(*module, blob_value->getType(), true,
+                                                 llvm::GlobalValue::ExternalLinkage, blob_value,
+                                                 runtime::symbol::tvm_dev_mblob, nullptr,
+                                                 llvm::GlobalVariable::NotThreadLocal, 0);
+
+#if TVM_LLVM_VERSION >= 100
+  tvm_dev_mblob->setAlignment(llvm::Align(1));
+#else
+  tvm_dev_mblob->setAlignment(1);
+#endif
+
+  if (target_triple.isOSWindows()) {
+    tvm_dev_mblob->setDLLStorageClass(llvm::GlobalVariable::DLLExportStorageClass);
+  }
+
+  if (system_lib) {
+    // LLVM type helper
+    auto void_ty = llvm::Type::getVoidTy(*ctx);
+    auto int32_ty = llvm::Type::getInt32Ty(*ctx);
+    auto int8_ty = llvm::Type::getInt8Ty(*ctx);
+    auto int8_ptr_ty = int8_ty->getPointerTo(0);
+
+    llvm::Constant* constant_zero = llvm::Constant::getNullValue(int32_ty);
+    auto* tvm_dev_mblob_reg =
+        new llvm::GlobalVariable(*module, int32_ty,
+                                 false, llvm::GlobalValue::InternalLinkage,
+                                 constant_zero,
+                                 std::string(runtime::symbol::tvm_dev_mblob) + "_reg_");
+    auto tvm_dev_mblob_reg_alignment = module->getDataLayout().getABITypeAlignment(int32_ty);
+#if TVM_LLVM_VERSION >= 100
+    tvm_dev_mblob_reg->setAlignment(llvm::Align(tvm_dev_mblob_reg_alignment));
+#else
+    tvm_dev_mblob_reg->setAlignment(tvm_dev_mblob_reg_alignment);
+#endif
+
+    auto* tvm_dev_mblob_string_ty =
+        llvm::ArrayType::get(int8_ty, std::strlen(runtime::symbol::tvm_dev_mblob) + 1);
+    auto* tvm_dev_mblob_string_value =
+        llvm::ConstantDataArray::getString(*ctx, runtime::symbol::tvm_dev_mblob, true);
+    auto* tvm_dev_mblob_string =
+        new llvm::GlobalVariable(*module, tvm_dev_mblob_string_ty,
+                                 true, llvm::GlobalValue::PrivateLinkage,
+                                 tvm_dev_mblob_string_value,
+                                 std::string(runtime::symbol::tvm_dev_mblob) + ".str");
+#if TVM_LLVM_VERSION >= 100
+    tvm_dev_mblob_string->setAlignment(llvm::Align(1));
+#else
+    tvm_dev_mblob_string->setAlignment(1);
+#endif
+
+    // Global init function
+    llvm::Function* init_fn = llvm::Function::Create(llvm::FunctionType::get(void_ty, false),
+                                                     llvm::GlobalValue::InternalLinkage,
+                                                     llvm::Twine("_GLOBAL__sub_I_", module_name),
+                                                     module.get());
+
+    // Create variable initialization function.
+    llvm::Function* var_init_fn = llvm::Function::Create(llvm::FunctionType::get(void_ty, false),
+                                                         llvm::GlobalValue::InternalLinkage,
+                                                         llvm::Twine("__cxx_global_var_init"),
+                                                         module.get());
+
+    // Create TVMBackendRegisterSystemLibSymbol function
+    llvm::Function* tvm_backend_fn =
+        llvm::Function::Create(llvm::FunctionType::get(int32_ty, {int8_ptr_ty, int8_ptr_ty}, false),
+                               llvm::GlobalValue::ExternalLinkage,
+                               llvm::Twine("TVMBackendRegisterSystemLibSymbol"),
+                               module.get());
+
+    // Set necessary fn sections
+    auto get_static_init_section_specifier = [&target_triple]() -> std::string {
+       if (target_triple.isOSLinux()) {
+         return ".text.startup";
+       } else if (target_triple.isOSDarwin()) {
+         return "__TEXT,__StaticInit,regular,pure_instructions";
+       } else {
+         return "";
+       }
+    };
+
+    auto static_init_section_specifier = get_static_init_section_specifier();
+
+    if (!static_init_section_specifier.empty()) {
+      init_fn->setSection(static_init_section_specifier);
+      var_init_fn->setSection(static_init_section_specifier);
+    }
+
+
 
 Review comment:
   Will fix.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364573509
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Sorry for missing your comment before. I find one strange thing of this. When I do this:
   ```
     std::string bin = SerializeModule(mod);  
     std::string header;
     for (size_t i = 0; i < sizeof(nbytes); ++i) {
       header.push_back(((nbytes >> (i * 8)) & 0xffUL));
     }
    std::string blob = header + bin;
    std::cout << "blob " << blob; // Could print content
    
     // Call codegen_blob to generate LLVM module
     std::string codegen_f_name = "codegen.codegen_blob";
     // the codegen function.
     const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
     CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
     runtime::Module m = (*codegen_f)(blob, system_lib, target);
     return m;
   ```
   Inside the  codegen.codegen_blob function, after accepting the blob string data. Can not print blob. Do I miss something?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364338087
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Do we need to pass hex here? can we directly pass the binary blob to LLVM?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365069861
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   ```python
   create_shared.get_target_triple = get_target_by_dump_machine(
        "g++" if sys.platform == "darwin" or sys.platform.startswith("linux") else None)
   
   ndk.get_target_triple = get_target_by_dump_machine(NDK_CC)
   ```
   Note how ```get_target_by_dump_machine``` takes the compiler argument and returns a closure impl of get_target_triple

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365066925
 
 

 ##########
 File path: python/tvm/contrib/ndk.py
 ##########
 @@ -64,5 +64,28 @@ def create_shared(output,
         msg += py_str(out)
         raise RuntimeError(msg)
 
+def get_target_triple():
 
 Review comment:
   by having the functor, we can reuse get_target_triple function among ndk and cc

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572372688
 
 
   if cl is used, I think we can safely assume the target is windows, we only need to know whether if it is win32 or win64, i am not that familar with cl to know for sure, but i guess in that case we can pass and use LLVM module detection, or fallback to c

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364913418
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   please also add support for cross_compiler in this file

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364951790
 
 

 ##########
 File path: src/codegen/llvm/codegen_blob.h
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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 codegen_blob.h
+ * \brief Code Generation of blob data
+ */
+#ifndef TVM_CODEGEN_LLVM_CODEGEN_BLOB_H_
+#define TVM_CODEGEN_LLVM_CODEGEN_BLOB_H_
+#ifdef TVM_LLVM_VERSION
+#include <utility>
+#include <memory>
+#include <string>
+#include "llvm_common.h"
+
+namespace tvm {
+namespace codegen {
+/**
+ * \brief Code Generation of blob data
+ *
+ * \param data Blob data
+ * \param system_lib Whether expose as system library.
+ * \param target LLVM target
+ *
+ * \return LLVM module and LLVM context
+ */
+std::pair<std::unique_ptr<llvm::Module>,
+          std::shared_ptr<llvm::LLVMContext>> CodeGenBlob(const std::string& data,
+                                                          bool system_lib,
+                                                          const std::string& target);
 
 Review comment:
   target_triple?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364550096
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   std::string header;
   for (size_t i = 0; i < sizeof(nbytes); ++i) {
       header.push_back(((nbytes >> (i * 8)) & 0xffUL));
   }
   then pass header + content

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364360657
 
 

 ##########
 File path: include/tvm/build_module.h
 ##########
 @@ -109,6 +109,23 @@ class Target : public ObjectRef {
    */
   TVM_DLL static tvm::Target Current(bool allow_not_defined = true);
 
+  /*!
+   * \brief Get the target host based on LLVM.
+   * \return The target host based on LLVM.
+   */
 
 Review comment:
   I am not sure, but it looks a little wired to me to have these in `Target`. Shouldn't we just use `Target::llvm()` directly? 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365062086
 
 

 ##########
 File path: src/codegen/llvm/codegen_blob.h
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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 codegen_blob.h
+ * \brief Code Generation of blob data
+ */
+#ifndef TVM_CODEGEN_LLVM_CODEGEN_BLOB_H_
+#define TVM_CODEGEN_LLVM_CODEGEN_BLOB_H_
+#ifdef TVM_LLVM_VERSION
+#include <utility>
+#include <memory>
+#include <string>
+#include "llvm_common.h"
+
+namespace tvm {
+namespace codegen {
+/**
+ * \brief Code Generation of blob data
+ *
+ * \param data Blob data
+ * \param system_lib Whether expose as system library.
+ * \param target LLVM target
+ *
+ * \return LLVM module and LLVM context
+ */
+std::pair<std::unique_ptr<llvm::Module>,
+          std::shared_ptr<llvm::LLVMContext>> CodeGenBlob(const std::string& data,
+                                                          bool system_lib,
+                                                          const std::string& target);
 
 Review comment:
   Will fix.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364365522
 
 

 ##########
 File path: src/codegen/llvm/codegen_blob.cc
 ##########
 @@ -0,0 +1,215 @@
+/*
+ * 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 codegen_blob.cc
+ */
+#ifdef TVM_LLVM_VERSION
+#include <tvm/runtime/module.h>
+#include <cstring>
+#include <cstdint>
+#include "codegen_blob.h"
+
 
 Review comment:
   no need this blank line

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365092221
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,10 +51,41 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_by_dump_machine(compiler):
+    """ Functor of get_target_triple that can get the target triple using compiler.
+
+    Parameters
+    ----------
+    compiler : Optional[str]
+        The compiler.
+
+    Returns
+    -------
+    out: Callable
+        A function that can get target triple according to dumpmachine option of compiler.
+    """
+    def get_target_triple():
+        """ Get target triple according to dumpmachine option of compiler."""
+        if compiler:
+            cmd = [compiler, "-dumpmachine"]
+            proc = subprocess.Popen(
+                cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+            (out, _) = proc.communicate()
+            if proc.returncode != 0:
+                msg = "dumpmachine error:\n"
+                msg += py_str(out)
+                raise None
 
 Review comment:
   My bad. I just wanted to return None. Will fix now.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365066395
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   I specificlaly mean add support here https://github.com/apache/incubator-tvm/blob/master/python/tvm/contrib/cc.py#L107

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572378913
 
 
   sounds good @FrozenGene  i agree with all your points

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364364907
 
 

 ##########
 File path: src/codegen/build_module.cc
 ##########
 @@ -882,6 +887,16 @@ TVM_REGISTER_GLOBAL("_GetCurrentTarget")
   *ret = Target::Current(allow_not_defined);
   });
 
+TVM_REGISTER_GLOBAL("_GetLLVMTargetHost")
+.set_body([](TVMArgs args, TVMRetValue* ret) {
+  *ret = Target::GetLLVMTargetHost();
+});
+
+TVM_REGISTER_GLOBAL("_SetLLVMTargetHost")
 
 Review comment:
   Yeah, I don't think this is necessary. I have some comments above, please see if they make sense. We know exactly where we need LLVM target so we can probably just directly use it, instead of using these wrappers which seem unsafe to me as well.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364397872
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   I think we need this. The most important reason is we need this information:
   ```
     for (size_t i = 0; i < sizeof(nbytes); ++i) {
       os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
     }
   ```
   Because when we load it back, we will do like this:
   ```
     uint64_t nbytes = 0;
     for (size_t i = 0; i < sizeof(nbytes); ++i) {
       uint64_t c = mblob[i];
       nbytes |=  (c & 0xffUL) << (i * 8);
     }
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364578922
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Cool!

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365065474
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
 
 Review comment:
   Will do

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572232368
 
 
   Hmm, i meant to have someway to automatically discover the triple and pass around, let us think a bit more about it. For example, if there is already an LLVM module in the modules, then we can get that from that module. 
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364338819
 
 

 ##########
 File path: src/codegen/build_module.cc
 ##########
 @@ -882,6 +887,16 @@ TVM_REGISTER_GLOBAL("_GetCurrentTarget")
   *ret = Target::Current(allow_not_defined);
   });
 
+TVM_REGISTER_GLOBAL("_GetLLVMTargetHost")
+.set_body([](TVMArgs args, TVMRetValue* ret) {
+  *ret = Target::GetLLVMTargetHost();
+});
+
+TVM_REGISTER_GLOBAL("_SetLLVMTargetHost")
 
 Review comment:
   It is a bit bad to use global state to pass this argument around. Would be great to pass this argument as an argument of PackToLLVM instead.
   
   
   I wonder if it is necessary to set the target to be the specific target, or can we directly use a somewhat generic target (this means LLVM won't perform any target specific optimization which is fine here)

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364573509
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Sorry for missing your comment before. I find one strange thing of this. When I do this:
   ```
     std::string bin = SerializeModule(mod);  
     std::string header;
     for (size_t i = 0; i < sizeof(nbytes); ++i) {
       header.push_back(((nbytes >> (i * 8)) & 0xffUL));
     }
    std::string blob = header + bin;
    std::cout << "blob " << blob; // Could print content
    
     // Call codegen_blob to generate LLVM module
     std::string codegen_f_name = "codegen.codegen_blob";
     // the codegen function.
     const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
     CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
     runtime::Module m = (*codegen_f)(blob, system_lib, target);
     return m;
   ```
   Inside the  codegen.codegen_blob function, we can accept the blob string data. Print the blob is empty. But if we use previous os << std::hex to get hex data, we could accept the data successfully. Do I miss something?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365069861
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   ```python
   create_shared.get_target_triple = get_target_by_dump_machine(
        "g++" if sys.platform == "darwin" or sys.platform.startswith("linux") else None)
   
   ndk.get_target_triple = get_target_by_dump_machine(NDK_CC)
   
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics merged pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics merged pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657
 
 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364549582
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   This is only  a way to serialize the nbytes in a fixed endian. We should be able to directly append it to binary string and put it in the beginning.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365067929
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   > https://github.com/apache/incubator-tvm/blob/master/python/tvm/contrib/cc.py#L107
   
   Ok. combine your context I think you mean:
   
   ```python
   def get_target_by_dump_machine(compiler="g++"): # default value be g++. could be specified by users
       def get_target_triple():
       if sys.platform == "darwin" or sys.platform.startswith("linux"):
           cmd = [compiler, "-dumpmachine"] # change to compiler
           proc = subprocess.Popen(
               cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
           (out, _) = proc.communicate()
           if proc.returncode != 0:
               msg = "dumpmachine error:\n"
               msg += py_str(out)
               raise RuntimeError(msg)
           return py_str(out)
       elif sys.platform == "win32":
           return None
       else:
           raise ValueError("Unsupported platform")
   return get_target_triple
   # lazily create
   create_shared.get_target_by_dump_machine = get_target_by_dump_machine
   def cross_compiler():
        ...
       fcompile.get_target_by_dump_machine = get_target_by_dump_machine
   ```
   
   Also we need to add this support for NDK and `build_create_shared_func` so that all fcompile could have this fn support.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364363249
 
 

 ##########
 File path: python/tvm/build_module.py
 ##########
 @@ -632,6 +632,10 @@ def build(inputs,
     if not target_host:
         target_host = "llvm" if module.enabled("llvm") else "stackvm"
 
+    # set the target host based on llvm
 
 Review comment:
   do we really need to set target here? can we just pass `target.create("llvm")` at line 145 of `module.py` when it is enabled? Then we can remove a lot of code that are rarely/never used at any other places.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364364002
 
 

 ##########
 File path: src/codegen/build_module.cc
 ##########
 @@ -219,7 +219,8 @@ std::string GetDeviceName(const std::string& target_str) {
 
   return "";
 }
-
+// Initialize static member of target host
+Target Target::llvm_target_host_;
 
 Review comment:
   I don't think we need 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365060564
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   > please also add support for cross_compiler in this file
   
   In my mind, I think get_target_triple should be used for host running TVM or we call ndk (we could know the compiler according to TVM_NDK_CC). Like we use cross_compiler (currently be used by MicroDev) to generate C file, we don't know the compiler. If I don't understand correctly of cross_compiler's effect, please correct me. Thanks!

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364573509
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Sorry for missing your comment before. I find one strange thing of this. When I do this:
   ```
     std::string bin = SerializeModule(mod);  
     std::string header;
     for (size_t i = 0; i < sizeof(nbytes); ++i) {
       header.push_back(((nbytes >> (i * 8)) & 0xffUL));
     }
    std::string blob = header + bin;
    std::cout << "blob " << blob; // Could print content
    
     // Call codegen_blob to generate LLVM module
     std::string codegen_f_name = "codegen.codegen_blob";
     // the codegen function.
     const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
     CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
     runtime::Module m = (*codegen_f)(blob, system_lib, target);
     return m;
   ```
   Inside the  codegen.codegen_blob function, we can't accept the blob string data. Print the blob is empty. But if we use previous os << std::hex to get hex data, we could accept the data successfully. Do I miss something?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572375077
 
 
   > > if cl is used, I think we can safely assume the target is windows, we only need to know whether if it is win32 or win64, i am not that familar with cl to know for sure, but i guess in that case we can pass and use LLVM module detection, or fallback to c
   > 
   > Yes. We could assume the target is Windows for sure. However, i think besides we need to know win32 / win64, one more thing I am a little worried. Windows could run ARM CPU (`cl.exe` could run Windows 10 on ARM? I can not make sure yet too).
   
   Ah...One tricky way I just see on the YoutuBe...
   ![image](https://user-images.githubusercontent.com/7287321/72037208-42026780-32d8-11ea-902a-5048249e4cfa.png)
   Maybe we could use `DUMPBIN` of `cl.exe` to make sure it is ARM or x86...But, it is so tricky, I personally think we maybe could fallback to c on Windows. Because I think
   ```
   If the module lists already contains an LLVM module, we can get triples from those modules
   ```
   could solve most of our cases.
   
   How about your opinion?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572361186
 
 
   > Some ideas about automatic detection, in the order of things that can be tried
   > 
   > * If the module lists already contains an LLVM module, we can get triples from those modules
   > * The property could be part of fcompile.get_target_triple()
   >   
   >   * Use hasattr to detect if fcompile contains the property
   >   * The function can return None, which means unable to detect
   >   * Note that for gcc and clang `gcc -dumpmachine` will give you the triple
   >   * Add support for this function in create_shared(using `gcc -dumpmachine`)
   > * If we cannot detect using the above approach,  fallback to PackToC, note that this is super unlikely
   
   Good idea. One quick question, how about Windows's compiler `cl`? The `cl` (https://docs.microsoft.com/en-us/cpp/build/reference/compiler-options-listed-by-category?view=vs-2019) seems doesn't have option to show arch information of its self.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572236081
 
 
   Some ideas about automatic detection, in the order of things that can be tried
   
   - If the module lists already contains an LLVM module, we can get triples from those modules
   - The property could be part of fcompile.get_target_triple()
        - Use hasattr to detect if fcompile contains the property
        - The function can return None, which means unable to detect
        - Note that for gcc and clang ```gcc -dumpmachine``` will give you the triple
        - Add support for this function in create_shared(using ```gcc -dumpmachine```)
   - If we cannot detect using the above approach,  fallback to PackToC, note that this is super unlikely
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364914242
 
 

 ##########
 File path: src/codegen/llvm/llvm_module.cc
 ##########
 @@ -62,6 +63,11 @@ class LLVMModuleNode final : public runtime::ModuleNode {
       return PackedFunc([flag](TVMArgs args, TVMRetValue *rv) {
           * rv = flag;
         });
+    } else if (name == "get_target_triple") {
 
 Review comment:
   consider rename to _get_target_triple just to avoid potential user defined function. 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364573509
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Sorry for missing your comment before. I find one strange thing of this. When I do this:
   ```
     std::string bin = SerializeModule(mod);  
     std::string header;
     for (size_t i = 0; i < sizeof(nbytes); ++i) {
       header.push_back(((nbytes >> (i * 8)) & 0xffUL));
     }
    std::string blob = header + bin;
    std::cout << "blob " << blob; // Could print content
    
     // Call codegen_blob to generate LLVM module
     std::string codegen_f_name = "codegen.codegen_blob";
     // the codegen function.
     const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
     CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
     runtime::Module m = (*codegen_f)(blob, system_lib, target);
     return m;
   ```
   Inside the  codegen.codegen_blob function, we can't accept the blob string data. Can not print blob. Do I miss something?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364923528
 
 

 ##########
 File path: src/codegen/llvm/codegen_blob.cc
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * 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 codegen_blob.cc
+ */
+#ifdef TVM_LLVM_VERSION
+#include <tvm/runtime/module.h>
+#include <cstring>
+#include "codegen_blob.h"
+
+namespace tvm {
+namespace codegen {
+
+std::pair<std::unique_ptr<llvm::Module>,
+          std::shared_ptr<llvm::LLVMContext>> CodeGenBlob(const std::string& data,
+                                                          bool system_lib,
+                                                          const std::string& target) {
+  InitializeLLVM();
+  auto tm = GetLLVMTargetMachine(std::string("-target ") + target);
+  auto target_triple = tm->getTargetTriple();
+  auto ctx = std::make_shared<llvm::LLVMContext>();
+  std::string module_name = "devc";
+  std::unique_ptr<llvm::Module> module(new llvm::Module(module_name, *ctx));
+  module->setTargetTriple(target_triple.str());
+  module->setDataLayout(tm->createDataLayout());
+  auto* blob_value = llvm::ConstantDataArray::getString(*ctx, data, false);
+  auto* tvm_dev_mblob = new llvm::GlobalVariable(*module, blob_value->getType(), true,
+                                                 llvm::GlobalValue::ExternalLinkage, blob_value,
+                                                 runtime::symbol::tvm_dev_mblob, nullptr,
+                                                 llvm::GlobalVariable::NotThreadLocal, 0);
+
+#if TVM_LLVM_VERSION >= 100
+  tvm_dev_mblob->setAlignment(llvm::Align(1));
+#else
+  tvm_dev_mblob->setAlignment(1);
+#endif
+
+  if (target_triple.isOSWindows()) {
+    tvm_dev_mblob->setDLLStorageClass(llvm::GlobalVariable::DLLExportStorageClass);
+  }
+
+  if (system_lib) {
+    // LLVM type helper
+    auto void_ty = llvm::Type::getVoidTy(*ctx);
+    auto int32_ty = llvm::Type::getInt32Ty(*ctx);
+    auto int8_ty = llvm::Type::getInt8Ty(*ctx);
+    auto int8_ptr_ty = int8_ty->getPointerTo(0);
+
+    llvm::Constant* constant_zero = llvm::Constant::getNullValue(int32_ty);
+    auto* tvm_dev_mblob_reg =
+        new llvm::GlobalVariable(*module, int32_ty,
+                                 false, llvm::GlobalValue::InternalLinkage,
+                                 constant_zero,
+                                 std::string(runtime::symbol::tvm_dev_mblob) + "_reg_");
+    auto tvm_dev_mblob_reg_alignment = module->getDataLayout().getABITypeAlignment(int32_ty);
+#if TVM_LLVM_VERSION >= 100
+    tvm_dev_mblob_reg->setAlignment(llvm::Align(tvm_dev_mblob_reg_alignment));
+#else
+    tvm_dev_mblob_reg->setAlignment(tvm_dev_mblob_reg_alignment);
+#endif
+
+    auto* tvm_dev_mblob_string_ty =
+        llvm::ArrayType::get(int8_ty, std::strlen(runtime::symbol::tvm_dev_mblob) + 1);
+    auto* tvm_dev_mblob_string_value =
+        llvm::ConstantDataArray::getString(*ctx, runtime::symbol::tvm_dev_mblob, true);
+    auto* tvm_dev_mblob_string =
+        new llvm::GlobalVariable(*module, tvm_dev_mblob_string_ty,
+                                 true, llvm::GlobalValue::PrivateLinkage,
+                                 tvm_dev_mblob_string_value,
+                                 std::string(runtime::symbol::tvm_dev_mblob) + ".str");
+#if TVM_LLVM_VERSION >= 100
+    tvm_dev_mblob_string->setAlignment(llvm::Align(1));
+#else
+    tvm_dev_mblob_string->setAlignment(1);
+#endif
+
+    // Global init function
+    llvm::Function* init_fn = llvm::Function::Create(llvm::FunctionType::get(void_ty, false),
+                                                     llvm::GlobalValue::InternalLinkage,
+                                                     llvm::Twine("_GLOBAL__sub_I_", module_name),
+                                                     module.get());
+
+    // Create variable initialization function.
+    llvm::Function* var_init_fn = llvm::Function::Create(llvm::FunctionType::get(void_ty, false),
+                                                         llvm::GlobalValue::InternalLinkage,
+                                                         llvm::Twine("__cxx_global_var_init"),
+                                                         module.get());
+
+    // Create TVMBackendRegisterSystemLibSymbol function
+    llvm::Function* tvm_backend_fn =
+        llvm::Function::Create(llvm::FunctionType::get(int32_ty, {int8_ptr_ty, int8_ptr_ty}, false),
+                               llvm::GlobalValue::ExternalLinkage,
+                               llvm::Twine("TVMBackendRegisterSystemLibSymbol"),
+                               module.get());
+
+    // Set necessary fn sections
+    auto get_static_init_section_specifier = [&target_triple]() -> std::string {
+       if (target_triple.isOSLinux()) {
+         return ".text.startup";
+       } else if (target_triple.isOSDarwin()) {
+         return "__TEXT,__StaticInit,regular,pure_instructions";
+       } else {
+         return "";
+       }
+    };
+
+    auto static_init_section_specifier = get_static_init_section_specifier();
+
+    if (!static_init_section_specifier.empty()) {
+      init_fn->setSection(static_init_section_specifier);
+      var_init_fn->setSection(static_init_section_specifier);
+    }
+
+
 
 Review comment:
   remove this blank line

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572236081
 
 
   Some ideas about automatic detection, in the order of things that can be tried
   
   - If the module lists already contains an LLVM module, we can get triples from those modules
   - The property could be part of fcompile.target_triple(), if such attrs existed
        - The function can return None, which means unable to detect
        - Note that for gcc and clang ```gcc -dumpmachine``` will give you the triple
   - If we cannot detect using the above approach,  fallback to PackToC, note that this is super unlikely
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364577848
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   Sorry I forget to mention you cannot pass std string because that will be treated as null terminate c str. Pass a ByteArray instead 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364912655
 
 

 ##########
 File path: python/tvm/contrib/ndk.py
 ##########
 @@ -64,5 +64,28 @@ def create_shared(output,
         msg += py_str(out)
         raise RuntimeError(msg)
 
+def get_target_triple():
 
 Review comment:
   Let us create a functor that returns this function and reuse it in most places
   ```
   def get_target_by_dump_machine(compiler):
        def get_target_triple():
            ...
        return get_target_triple
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364576391
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   After printing the length, inside the PackImportToLLVM, blob is 1044, but inside codegen.codegen_blob, the length of blob is 2.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572199993
 
 
   @tqchen @zhiics About the LLVM target part, I also think current way maybe is not good. I also think of it and cost my some time. So I want to discuss it too. 
   
   Firstly Let me describe why we need llvm target host. 
   
   When we use PackImportToLLVM, we will need to use LLVM to create LLVM module. 
   
   However, we must need one target machine, this is answered to @tqchen 's question. We can not create one generic target. Because we have platform's specific feature to handle. See:https://github.com/apache/incubator-tvm/pull/4657/files#diff-f3a67dc7be877c1da11892f6a8e5ae80R107-R109 and https://github.com/apache/incubator-tvm/pull/4657/files#diff-f3a67dc7be877c1da11892f6a8e5ae80R107-R109. Another thing is if we leave the target is empty, i.e. no module->setTargetTriple(target_triple.str()); On llvm 6.0 of Mac will report problem : assert error Target-incompatible DataLayout.
   
   More import thing is we need to consider the target host is not x86 cpu. For example, target host is llvm -target aarch64, if we leave it empty and build it on x86 server, we will generate devc.o into x86 format, but in fact we want aarch64 of devc.o. So in the codegen_blob part, we should know the correct llvm target host to generate correct file.
   
   Current way is a little ugly I think too. Current way will create target host based on LLVM when we have detected target host finally (because tvm.build / relay.build's api could leave `target_host` be None, we have some logic to handle this situation). One simple way of course is let users specific the target host of llvm representation, but I use this ugly way is just to detect it automatically and don't want to let users do. I also want to listen to your advices. Thanks in advance.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364911682
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
 
 Review comment:
   alignment 
   ```
   Returns
   -------
   out: Optional[str] 
       The result target triple
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365067929
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   > https://github.com/apache/incubator-tvm/blob/master/python/tvm/contrib/cc.py#L107
   
   Ok. combine your context I think you mean:
   
   ```python
   def get_target_by_dump_machine(compiler="g++"): # default value be g++. could be specified by other api
       def get_target_triple():
       if sys.platform == "darwin" or sys.platform.startswith("linux"):
           cmd = [compiler, "-dumpmachine"] # change to compiler
           proc = subprocess.Popen(
               cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
           (out, _) = proc.communicate()
           if proc.returncode != 0:
               msg = "dumpmachine error:\n"
               msg += py_str(out)
               raise RuntimeError(msg)
           return py_str(out)
       elif sys.platform == "win32":
           return None
       else:
           raise ValueError("Unsupported platform")
   return get_target_triple
   # lazily create
   create_shared.get_target_by_dump_machine = get_target_by_dump_machine
   def cross_compiler():
        ...
       fcompile.get_target_by_dump_machine = get_target_by_dump_machine
   ```
   
   Also we need to add this support for NDK and `build_create_shared_func` so that all fcompile could have this fn support.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364913289
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
 
 Review comment:
   Please also add wrapper for cross compiler

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572236081
 
 
   Some ideas about automatic detection, in the order of things that can be tried
   
   - If the module lists already contains an LLVM module, we can get triples from those modules
   - The property could be part of fcompile.target_triple(), if such attrs existed
        - The function can return None, which means unable to detect
        - Note that for gcc and clang ```gcc -dumpmachine``` will give you the triple
   - If we cannot detect using the above approach, fallback to PackToC, note that this is super unlikely
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572373834
 
 
   > if cl is used, I think we can safely assume the target is windows, we only need to know whether if it is win32 or win64, i am not that familar with cl to know for sure, but i guess in that case we can pass and use LLVM module detection, or fallback to c
   
   Yes. We could assume the target is Windows for sure. However, i think besides we need to know win32 / win64, one more thing I am a little worried. Windows could run ARM CPU (`cl.exe` could run Windows 10 on ARM? I can not make sure yet too).

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572236081
 
 
   Some ideas about automatic detection, in the order of things that can be tried
   
   - If the module lists already contains an LLVM module, we can get triples from those modules
   - The property could be part of fcompile.get_target_triple(), if such attrs existed
        - The function can return None, which means unable to detect
        - Note that for gcc and clang ```gcc -dumpmachine``` will give you the triple
   - If we cannot detect using the above approach,  fallback to PackToC, note that this is super unlikely
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-573118852
 
 
   Thanks @FrozenGene @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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365069096
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   > Sorry I specificlaly mean add support here https://github.com/apache/incubator-tvm/blob/master/python/tvm/contrib/cc.py#L107
   > 
   > By allowing passing in an optional get_target_triple function
   
   Sorry I just saw your recent change of `By allowing passing in an optional get_target_triple function`. I understood previous that we should only expose `get_target_by_dump_machine` and `get_target_triple` should only accept `compiler` parameter. Seems that you want we could accept one functor of `get_target_triple` in cross_compiler so that we could get this information, i.e.
   
   ```python
   def get_target_by_dump_machine(compiler="g++"): # default value be g++. could be specified by other api
       def get_target_triple():
       if sys.platform == "darwin" or sys.platform.startswith("linux"):
           cmd = [compiler, "-dumpmachine"] # change to compiler
           proc = subprocess.Popen(
               cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
           (out, _) = proc.communicate()
           if proc.returncode != 0:
               msg = "dumpmachine error:\n"
               msg += py_str(out)
               raise RuntimeError(msg)
           return py_str(out)
       elif sys.platform == "win32":
           return None
       else:
           raise ValueError("Unsupported platform")
   
   def cross_compiler(compile_func, base_options=None, output_format="so",  get_target_triple=None):
    ...
   _fcompile.get_target_by_dump_machine = get_target_triple if get_target_triple else get_target_by_dump_machine
   ```

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365065825
 
 

 ##########
 File path: include/tvm/codegen.h
 ##########
 @@ -59,6 +59,21 @@ runtime::Module Build(const Array<LoweredFunc>& funcs,
  * \return cstr The C string representation of the file.
  */
 std::string PackImportsToC(const runtime::Module& m, bool system_lib);
+
+/*!
+ * \brief Pack imported device library to a LLVM module.
+ *  Compile the LLVM module and link with the host library
+ *  will allow the DSO loader to automatically discover and import
+ *  the dependency from the shared library.
+ *
+ * \param m The host module with the imports.
+ * \param system_lib Whether expose as system library.
+ * \param target LLVM target
 
 Review comment:
   Will fix

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572375077
 
 
   > > if cl is used, I think we can safely assume the target is windows, we only need to know whether if it is win32 or win64, i am not that familar with cl to know for sure, but i guess in that case we can pass and use LLVM module detection, or fallback to c
   > 
   > Yes. We could assume the target is Windows for sure. However, i think besides we need to know win32 / win64, one more thing I am a little worried. Windows could run ARM CPU (`cl.exe` could run Windows 10 on ARM? I can not make sure yet too).
   
   Ah...One tricky way I just see on the YoutuBe...
   ![image](https://user-images.githubusercontent.com/7287321/72037208-42026780-32d8-11ea-902a-5048249e4cfa.png)
   Maybe we could use `DUMPBIN` of `cl.exe` to make sure it is ARM or x86...But, it is so tricky, I personally think we maybe could fallback to c on Windows if we have fallen into `fcompile.get_target_triple()`. Because I think
   ```
   If the module lists already contains an LLVM module, we can get triples from those modules
   ```
   could solve most of our cases.
   
   How about your opinion?

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365071448
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   Understand your meaning. Thanks. Will create new pr after I finish lunch. 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365069930
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   In the case of _fcompile, we can simply assign it to get_target_triple if user pass it in, or assign it to None

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364363748
 
 

 ##########
 File path: python/tvm/module.py
 ##########
 @@ -140,10 +140,16 @@ def export_library(self,
                              module.get_function("__tvm_is_system_module")())
 
         if self.imported_modules:
-            path_cc = temp.relpath("devc.cc")
-            with open(path_cc, "w") as f:
-                f.write(_PackImportsToC(self, is_system_lib))
-            files.append(path_cc)
+            if enabled("llvm"):
+                path_obj = temp.relpath("devc.o")
+                m = _PackImportsToLLVM(self, is_system_lib, str(_target.get_llvm_target_host()))
 
 Review comment:
   See my above comment as this is the only place we may need the getter

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572222539
 
 
   > If we indeed needs to pass target triple, perhaps the best way to do so is to pass that as an argument of PackImportToLLVM
   
   @tqchen  So, your opinion is let the users specify. The API will become this:
   
    ```
      def export_library(self,
                          file_name,
                          fcompile=None,
                          llvm_target=None,
                          **kwargs):
       if enabled("llvm"):
          path_obj = temp.relpath("devc.o")
          m = _PackImportsToLLVM(self, is_system_lib,  llvm_target)
   ```
   
   If users doesn't set `llvm_target`, we use the default value. (system target triple).
   
   if users want to compile into another os / another cpu, they have to do this:
   ```module.export_library(..., llvm_target="llvm -target aarch64-linux-gnu-none")``` now.
   
   Does it make sense to you?
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572199993
 
 
   @tqchen @zhiics About the LLVM target part, I also think current way maybe is not good. I also think of it and cost my some time. So I want to discuss it too. 
   
   Firstly Let me describe why we need llvm target host. 
   
   When we use PackImportToLLVM, we will need to use LLVM to create LLVM module. 
   
   However, we must need one target machine, this is answered to @tqchen 's question. We can not create one generic target. Because we have platform's specific feature to handle. See:https://github.com/apache/incubator-tvm/blob/4f1772c0598ae708477f457a993f5be52fa71eb9/src/codegen/llvm/codegen_blob.cc#L107 and https://github.com/apache/incubator-tvm/blob/4f1772c0598ae708477f457a993f5be52fa71eb9/src/codegen/llvm/codegen_blob.cc#L167. Another thing is if we leave the target is empty, i.e. no module->setTargetTriple(target_triple.str()); On llvm 6.0 of Mac will report problem : assert error Target-incompatible DataLayout.
   
   More import thing is we need to consider the target host is not x86 cpu. For example, target host is llvm -target aarch64, if we leave it empty and build it on x86 server, we will generate devc.o into x86 format, but in fact we want aarch64 of devc.o. So in the codegen_blob part, we should know the correct llvm target host to generate correct file.
   
   Current way is a little ugly I think too. Current way will create target host based on LLVM when we have detected target host finally (because tvm.build / relay.build's api could leave `target_host` be None, we have some logic to handle this situation). One simple way of course is let users specific the target host of llvm representation, but I use this ugly way is just to detect it automatically and don't want to let users do. I also want to listen to your advices. Thanks in advance.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364911422
 
 

 ##########
 File path: include/tvm/codegen.h
 ##########
 @@ -59,6 +59,21 @@ runtime::Module Build(const Array<LoweredFunc>& funcs,
  * \return cstr The C string representation of the file.
  */
 std::string PackImportsToC(const runtime::Module& m, bool system_lib);
+
+/*!
+ * \brief Pack imported device library to a LLVM module.
+ *  Compile the LLVM module and link with the host library
+ *  will allow the DSO loader to automatically discover and import
+ *  the dependency from the shared library.
+ *
+ * \param m The host module with the imports.
+ * \param system_lib Whether expose as system library.
+ * \param target LLVM target
 
 Review comment:
   target-> target_triple

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365091179
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -15,7 +15,7 @@
 # specific language governing permissions and limitations
 # under the License.
 """Util to invoke C/C++ compilers in the system."""
-# pylint: disable=invalid-name
+# pylint: disable=invalid-name, raising-bad-type
 
 Review comment:
   do not disbale rising-bad-type

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572222539
 
 
   > If we indeed needs to pass target triple, perhaps the best way to do so is to pass that as an argument of PackImportToLLVM
   
   So, your opinion is let the users specify. The API will become this:
   
    ```
      def export_library(self,
                          file_name,
                          fcompile=None,
                          llvm_target=None,
                          **kwargs):
       if enabled("llvm"):
          path_obj = temp.relpath("devc.o")
          m = _PackImportsToLLVM(self, is_system_lib,  llvm_target)
   ```
   
   If users doesn't set `llvm_target`, we use the default value. (system target triple).
   
   if users want to compile into another os / another cpu, they have to do this:
   ```module.export_library(..., llvm_target="llvm -target aarch64-linux-gnu-none")``` now.
   
   Does it make sense to you?
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572418800
 
 
   Now, we could have new elegant and simpler automatic detection of LLVM target triple (thanks @tqchen 's good idea). @tqchen @zhiics could you help to review new code again?  Thanks!

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365066395
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   Sorry I specificlaly mean add support here https://github.com/apache/incubator-tvm/blob/master/python/tvm/contrib/cc.py#L107
   
   By allowing passing in an optional get_target_triple function

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364913154
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   Let us create a functor that returns this function and reuse it in most places
   ```
   def get_target_by_dump_machine(compiler):
        def get_target_triple():
            ...
        return get_target_triple
   ```
   
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene edited a comment on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572375077
 
 
   > > if cl is used, I think we can safely assume the target is windows, we only need to know whether if it is win32 or win64, i am not that familar with cl to know for sure, but i guess in that case we can pass and use LLVM module detection, or fallback to c
   > 
   > Yes. We could assume the target is Windows for sure. However, i think besides we need to know win32 / win64, one more thing I am a little worried. Windows could run ARM CPU (`cl.exe` could run Windows 10 on ARM? I can not make sure yet too).
   
   Ah...One tricky way I just see on the YoutuBe...
   ![image](https://user-images.githubusercontent.com/7287321/72037208-42026780-32d8-11ea-902a-5048249e4cfa.png)
   Maybe we could use `DUMPBIN` of `cl.exe` to make sure it is ARM or x86...But, it is so tricky, I personally think we maybe could fallback to c on Windows if we have fallen into `fcompile.get_target_triple()`. Because I think
   ```
   If the module lists already contains an LLVM module, we can get triples from those modules
   ```
   could solve most of our cases.
   
   How about your opinion? @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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364549582
 
 

 ##########
 File path: src/codegen/codegen.cc
 ##########
 @@ -211,5 +220,30 @@ std::string PackImportsToC(const runtime::Module& mod, bool system_lib) {
      << "#endif\n";
   return os.str();
 }
+
+runtime::Module PackImportsToLLVM(const runtime::Module& mod,
+                                  bool system_lib,
+                                  const std::string& target) {
+  std::string bin = SerializeModule(mod);
+
+  std::ostringstream os;
+  uint64_t nbytes = bin.length();
+  os << std::hex;
+  for (size_t i = 0; i < sizeof(nbytes); ++i) {
+    os << std::setfill('0') << std::setw(2) << ((nbytes >> (i * 8)) & 0xffUL);
+  }
+  for (size_t i = 0; i < bin.length(); ++i) {
+    int c = bin[i];
+    os << std::setfill('0') << std::setw(2) << (c & 0xff);
+  }
+
+  // Call codegen_blob to generate LLVM module
+  std::string codegen_f_name = "codegen.codegen_blob";
+  // the codegen function.
+  const PackedFunc* codegen_f = runtime::Registry::Get(codegen_f_name);
+  CHECK(codegen_f != nullptr)  << "codegen.codegen_blob is not presented.";
+  return (*codegen_f)(os.str(), system_lib, target);
 
 Review comment:
   This is only about the way to serialize the bytes in a fixed endian. We should be able to directly append it to binary string and put it in the beginning.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
FrozenGene commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364391776
 
 

 ##########
 File path: python/tvm/build_module.py
 ##########
 @@ -632,6 +632,10 @@ def build(inputs,
     if not target_host:
         target_host = "llvm" if module.enabled("llvm") else "stackvm"
 
+    # set the target host based on llvm
 
 Review comment:
   See my follow comment of this, I think it should answer why we can not use ```target.create("llvm")``` simply.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r364912249
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,9 +51,32 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_triple():
+    """ Get the target triple using compiler.
+
+    Returns
+    -------
+        out: str (Linux / Mac) or None (Win32)
+    """
+    if sys.platform == "darwin" or sys.platform.startswith("linux"):
+        cmd = ["g++", "-dumpmachine"]
+        proc = subprocess.Popen(
+            cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+        (out, _) = proc.communicate()
+        if proc.returncode != 0:
+            msg = "dumpmachine error:\n"
+            msg += py_str(out)
+            raise RuntimeError(msg)
+        return py_str(out)
+    elif sys.platform == "win32":
+        return None
+    else:
+        raise ValueError("Unsupported platform")
+
 
 # assign so as default output format
 create_shared.output_format = "so" if sys.platform != "win32" else "dll"
+create_shared.get_target_triple = get_target_triple()
 
 Review comment:
   Let us do it lazily(otherwise user who import cc implicitly and does not have a proper compiler tool chain will get an error)
   
   create_shared.get_target_triple = get_target_triple

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572219346
 
 
   If we indeed needs to pass target triple, perhaps the best way to do so is to pass that as an argument of PackImportToLLVM

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#issuecomment-572161931
 
 
   cc @zhiics @yzhliu @ajtulloch  

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4657: [CodeGen] Generate blob use LLVM directly
URL: https://github.com/apache/incubator-tvm/pull/4657#discussion_r365091130
 
 

 ##########
 File path: python/tvm/contrib/cc.py
 ##########
 @@ -51,10 +51,41 @@ def create_shared(output,
     else:
         raise ValueError("Unsupported platform")
 
+def get_target_by_dump_machine(compiler):
+    """ Functor of get_target_triple that can get the target triple using compiler.
+
+    Parameters
+    ----------
+    compiler : Optional[str]
+        The compiler.
+
+    Returns
+    -------
+    out: Callable
+        A function that can get target triple according to dumpmachine option of compiler.
+    """
+    def get_target_triple():
+        """ Get target triple according to dumpmachine option of compiler."""
+        if compiler:
+            cmd = [compiler, "-dumpmachine"]
+            proc = subprocess.Popen(
+                cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT)
+            (out, _) = proc.communicate()
+            if proc.returncode != 0:
+                msg = "dumpmachine error:\n"
+                msg += py_str(out)
+                raise None
 
 Review comment:
   Raise None seems to be wrong, either raise an error or return None

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


With regards,
Apache Git Services