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 2021/03/18 00:28:57 UTC

[GitHub] [tvm] csullivan opened a new pull request #7686: [Texture support][Part 1] TIR, codegen and device runtime support

csullivan opened a new pull request #7686:
URL: https://github.com/apache/tvm/pull/7686


   This PR introduces 2d texture memory support for TE, TIR, OpenCL codegen, and the OpenCL Device API. 
   
   The lowering process is driven principally by the TextureFlatten TIR lowering pass which flattens multi-dimensional loads and stores to two dimensional accesses. The texture lowering is agnostic to the utilization of the texture memory, e.g. via `cache_read("texture")` as a temporary scratch pad, or from a runtime tir::Buffer Var of TextureType.
   
   **Layout**
   - The current lowering path assumes two dimensional textures will always have the lowest dimension of size 4 (RGBA); in this way, the use of packed layouts for activations and weights are required. In [Part 2] schedules are introduced using NCHW4c and OIHW4o to take advantage of the texture lowering path. 
   - Separate Nd to 2d lowering conventions are employed for activation and for weights. For activations all dimensions save the last (excluding the vector length 4 dimension) are packed into columns of the texture image; for weights all dimensions save the first are packed into rows of the texture image: 
   ```
   scope == "texture" :: [A, B, C, D, 4] -> [A*B*C, D, 4]
   scope == "texture:weight" :: [A, B, C, D] -> [A, B*C*D, 4]
   ```
   Additionally, if any other layout lowering is desired, one can apply logical shape changes via a te.compute in the topi compute definition to ensure the flattening occurs as desired.
   
   - These two lowering conventions are used in TIR lowering, the opencl device runtime, and in the graph runtime, and are implemented in "src/runtime/texture.h". 
   - 
   
   **Codegen**
   - OpenCL codegen of `write_image` and `read_image` on image2d_t's of type CL_FLOAT and CL_HALF types and RGBA channel order are supported. 
   - A pass is added in code generation to infer `__read_only` and `__write_only` access qualifiers for the utilized texture buffers, resulting in kernel signatures like, 
   
   ```
   __kernel void fused_nn_kernel0(__read_only image2d_t placeholder0,  ..., __write_only image2d_t compute,) 
   ```
   
   - To avoid extra texture reads, SSA intermediates are utilized when an explicit cache_read to local memory is not specified. 
   - Single element index into an RGBA texture read is supported to enable outer product style broadcasting.
   
   **Device runtime**
   - The device runtime supports allocating texture memory both as a temporal workspace and as a runtime data space. In the latter case, special invocation of AllocDataSpace with a memory_scope == "texture(:weight)" is required. Special memory scopes were added to the runtime in https://github.com/apache/tvm/pull/7488. 
   - Workspace allocations are handled via a set of idle texture pools which are grown to match the requested sizes. The strategy employed is to first pick the pool which requires the least amount of extra space beyond, and then to minimize the amount of wasted space that growing a two dimensional pool may incur. A similar approach is taken for the ahead of time graph runtime memory planner for data space allocations (see: PR#).
   - CopyFromTo support is expanded to handle the case of directly reading from / writing to image buffers from host. 
   
   
   RFC in progress, once posted I will add a link here.


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

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



[GitHub] [tvm] tqchen commented on pull request #7686: [Texture support][Part 1] TIR, codegen and device runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-801531288


   Thanks @csullivan can you send the src/runtime change as a separate PR? Also would be good to discuss whether a special runtime texture memory alloca API is needed now that we have explicit memory support. Perhaps can change to a packedfunc given packed call now supports returning ptr value


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

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



[GitHub] [tvm] csullivan commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-900805679


   @ZihengJiang many thanks for the feedback, see my changes and comments. I've also added many tests. Please take a look when you have a chance. 


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] tqchen commented on a change in pull request #7686: [Texture support][Part 1] TIR, codegen and device runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r596475663



##########
File path: include/tvm/ir/type.h
##########
@@ -181,6 +181,55 @@ class PointerType : public Type {
   TVM_DEFINE_OBJECT_REF_METHODS(PointerType, Type, PointerTypeNode);
 };
 
+/*!
+ * \brief Low-level texture type.
+ *
+ *  TextureType represents type hints in the TIR to be
+ *  passed to the final code generator.
+ *
+ *  TextureType should not occur in the high-level analysis.
+ *
+ * \sa TextureType
+ */
+class TextureTypeNode : public TypeNode {
+ public:
+  /*!
+   * \brief The base type of the texture.
+   */

Review comment:
       Given this is quite close to PointerType, perhaps we should think about instead introducing a scope flag (str for now) to the PointerType, instead of introducing a new type

##########
File path: src/runtime/thread_storage_scope.h
##########
@@ -57,6 +57,8 @@ enum class StorageRank {
   kWMMAMatrixB = 5,
   /*! \brief wmma scope memory of accumulator */
   kWMMAAccumulator = 6,
+  /*! \brief global scope texture memory */
+  kTexture = 7,

Review comment:
       I believe we do not need to introduce a special scope. Instead we can directly make texture a tag, for example: ```global.texture-weight```
   
   This would reduce special casing in thread hierachy calculations. We should actually also remove WMMA later

##########
File path: src/tir/transforms/texture_flatten.cc
##########
@@ -0,0 +1,330 @@
+/*
+ * 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 texture_flatten.cc
+ * \brief Flattens texture from multi-dimensional array to 2D buffer access
+ */
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/target_info.h>
+#include <tvm/te/operation.h>
+#include <tvm/tir/analysis.h>
+#include <tvm/tir/buffer.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/expr.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <unordered_map>
+#include <stack>
+
+#include "../../arith/ir_visitor_with_analyzer.h"
+#include "../../runtime/thread_storage_scope.h"
+#include "../../runtime/texture.h"
+#include "../ir/buffer_common.h"
+#include "arg_binder.h"
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+namespace {
+
+using runtime::IsTextureStorage;
+using runtime::DefaultTextureLayoutSeparator;
+using runtime::ApplyTexture2DFlattening;
+
+inline PrimExpr SimplifyOffset(const Array<PrimExpr>& shape, const Array<PrimExpr>& index) {
+  PrimExpr base = make_const(DataType::Int(32), 0);
+  ICHECK_EQ(shape.size(), index.size());
+  arith::Analyzer ana;
+  if (index.size() > 0) {
+    PrimExpr offset = index[0];
+    for (size_t i = 1; i < index.size(); ++i) {
+      offset = MergeMulMod(&ana, offset * shape[i] + index[i]);

Review comment:
       I am not sure we would need this(through buffer common). MergeMulMod is a quite legacy function that needs to be simplified later, and ideally we should simply rely on analzyer simplify and leave the Buffer impl as it is




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

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



[GitHub] [tvm] csullivan commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r632596678



##########
File path: include/tvm/ir/type.h
##########
@@ -181,6 +181,55 @@ class PointerType : public Type {
   TVM_DEFINE_OBJECT_REF_METHODS(PointerType, Type, PointerTypeNode);
 };
 
+/*!
+ * \brief Low-level texture type.
+ *
+ *  TextureType represents type hints in the TIR to be
+ *  passed to the final code generator.
+ *
+ *  TextureType should not occur in the high-level analysis.
+ *
+ * \sa TextureType
+ */
+class TextureTypeNode : public TypeNode {
+ public:
+  /*!
+   * \brief The base type of the texture.
+   */

Review comment:
       A storage specific type or scope is helpful in the IR so it can be tracked throughout various transformations including during [codegen](https://github.com/apache/tvm/pull/7686/files#diff-ee4125c7c00371317aff30351177693f0449409f24477f5f85cba7110d988616R216).




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

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



[GitHub] [tvm] csullivan commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-900805679


   @ZihengJiang many thanks for the feedback, see my changes and comments. I've also added many tests. Please take a look when you have a chance. 


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] ZihengJiang commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-903000978


   Merged. Thanks @csullivan !


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] ZihengJiang commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r683023953



##########
File path: src/target/source/codegen_opencl.cc
##########
@@ -27,18 +27,63 @@
 #include <vector>
 
 #include "../../runtime/opencl/opencl_module.h"
+#include "../../runtime/texture.h"
 #include "../../runtime/thread_storage_scope.h"
 #include "../build_common.h"
 
 namespace tvm {
 namespace codegen {
 
-CodeGenOpenCL::CodeGenOpenCL() { restrict_keyword_ = "restrict"; }
+class InferTextureAccess : public StmtExprVisitor {
+ public:
+  static constexpr const uint8_t read_access = 1;

Review comment:
       The naming convention is kReadAccess for constant.

##########
File path: src/tir/transforms/lower_tvm_builtin.cc
##########
@@ -98,6 +98,15 @@ class BuiltinLower : public StmtExprMutator {
     }
   }
 
+  Stmt VisitStmt_(const LetStmtNode* op) final {

Review comment:
       Why does mutation happens in `LetStmtNode` instead of `CallNode` directly?

##########
File path: src/target/source/codegen_c.h
##########
@@ -194,6 +194,8 @@ class CodeGenC : public ExprFunctor<void(const PrimExpr&, std::ostream&)>,
   virtual std::string CastFromTo(std::string value, DataType from, DataType target);
   // Get load of single element with expression
   virtual void PrintVecElemLoadExpr(DataType t, int i, const std::string& value, std::ostream& os);
+  // Print restrict keyword for a given Var if applicable
+  virtual void PrintRestrict(const Var& v, std::ostream& os);

Review comment:
       What does `restrict` mean 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] csullivan commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-902859990


   @tqchen Does this look good on your end?


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] csullivan commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-903002337


   Thanks @ZihengJiang, @tqchen, and @manupa-arm!


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] tqchen commented on a change in pull request #7686: [Texture support][Part 1] TIR, codegen and device runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r596475341



##########
File path: include/tvm/runtime/device_api.h
##########
@@ -181,7 +181,38 @@ class TVM_DLL DeviceAPI {
    * \param ptr The pointer to be freed.
    */
   virtual void FreeWorkspace(TVMContext ctx, void* ptr);
-
+  /*!
+   * \brief Allocate a two dimensional texture data space on device
+   * \param ctx The device context to perform operation.
+   * \param width The width of the 2d texture in elements
+   * \param height The height of the 2d texture in elements
+   * \param type_hint The type of elements.
+   */
+  virtual void* AllocTexture(TVMContext ctx, size_t width, size_t height, DLDataType type_hint = {});

Review comment:
       Now that we have the Special scope API, likely we do not want to expose this as a separate DeviceAPI




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

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



[GitHub] [tvm] junrushao1994 commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
junrushao1994 commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-903979835


   Congrats! Finally it's merged 🎉


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] csullivan commented on pull request #7686: [Texture support][Part 1] TIR, codegen and device runtime support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-801604284


   Thanks @tqchen. Yes I will break out the device runtime into its own PR. I plan to send an RFC in the next days.


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

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



[GitHub] [tvm] ZihengJiang commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r683023953



##########
File path: src/target/source/codegen_opencl.cc
##########
@@ -27,18 +27,63 @@
 #include <vector>
 
 #include "../../runtime/opencl/opencl_module.h"
+#include "../../runtime/texture.h"
 #include "../../runtime/thread_storage_scope.h"
 #include "../build_common.h"
 
 namespace tvm {
 namespace codegen {
 
-CodeGenOpenCL::CodeGenOpenCL() { restrict_keyword_ = "restrict"; }
+class InferTextureAccess : public StmtExprVisitor {
+ public:
+  static constexpr const uint8_t read_access = 1;

Review comment:
       The naming convention is kReadAccess for constant.




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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] tqchen commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-883670484


   cc @Lunderberg @masahi @ZihengJiang would be great if we can followup on this.


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r632398783



##########
File path: include/tvm/ir/type.h
##########
@@ -181,6 +181,55 @@ class PointerType : public Type {
   TVM_DEFINE_OBJECT_REF_METHODS(PointerType, Type, PointerTypeNode);
 };
 
+/*!
+ * \brief Low-level texture type.
+ *
+ *  TextureType represents type hints in the TIR to be
+ *  passed to the final code generator.
+ *
+ *  TextureType should not occur in the high-level analysis.
+ *
+ * \sa TextureType
+ */
+class TextureTypeNode : public TypeNode {
+ public:
+  /*!
+   * \brief The base type of the texture.
+   */

Review comment:
       @tqchen @csullivan , just curious (and for my understanding) why cant we point the Var to a buffer (using the buffer map) and buffer has a way to mention the scope ?
   
   Came here by looking at why this is needed in #8017 




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

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



[GitHub] [tvm] tqchen commented on pull request #7686: [Texture support][Part 1] TIR, codegen and device runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-801531561


   It would also be great to send an RFC describing the overall solution


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

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



[GitHub] [tvm] csullivan commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r690890097



##########
File path: src/tir/transforms/lower_tvm_builtin.cc
##########
@@ -98,6 +98,15 @@ class BuiltinLower : public StmtExprMutator {
     }
   }
 
+  Stmt VisitStmt_(const LetStmtNode* op) final {

Review comment:
       In this case I mutate the let
   ```
   let buffer = texture2d_alloca(...)
   ```
   to include stmts to check that the return value of the call is not nullptr, to include the free of the allocation in a sequence, and to check for non-zero return value of the free. This is behaving similar to the handling of the Allocate visitor.
   
   I'm quite open to alternatives if there is a better approach that I missed.

##########
File path: src/target/source/codegen_c.h
##########
@@ -194,6 +194,8 @@ class CodeGenC : public ExprFunctor<void(const PrimExpr&, std::ostream&)>,
   virtual std::string CastFromTo(std::string value, DataType from, DataType target);
   // Get load of single element with expression
   virtual void PrintVecElemLoadExpr(DataType t, int i, const std::string& value, std::ostream& os);
+  // Print restrict keyword for a given Var if applicable
+  virtual void PrintRestrict(const Var& v, std::ostream& os);

Review comment:
       This is the no-aliasing qualifier for opencl kernels. A compiler hint that pointer aliasing does not occur among the restricted pointer arguments.




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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] csullivan commented on pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#issuecomment-803199793


   I have split out the runtime support into a [Part 0] PR #7711. @ZihengJiang it contains a partial cherry pick from your #7629 and I would certainly appreciate your 👀 on it.
   
   @tqchen, I've posted an RFC with an overview of the approach, please see here: https://discuss.tvm.apache.org/t/rfc-texture-memory-support/9467. 
   
   Thanks for the suggestions on adding scope to PointerType, and also the suggestion on opting for a special global scope, I believe these can simplify portions of the code.


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

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



[GitHub] [tvm] csullivan commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
csullivan commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r690890097



##########
File path: src/tir/transforms/lower_tvm_builtin.cc
##########
@@ -98,6 +98,15 @@ class BuiltinLower : public StmtExprMutator {
     }
   }
 
+  Stmt VisitStmt_(const LetStmtNode* op) final {

Review comment:
       In this case I mutate the let
   ```
   let buffer = texture2d_alloca(...)
   ```
   to include stmts to check that the return value of the call is not nullptr, to include the free of the allocation in a sequence, and to check for non-zero return value of the free. This is behaving similar to the handling of the Allocate visitor.
   
   I'm quite open to alternatives if there is a better approach that I missed.

##########
File path: src/target/source/codegen_c.h
##########
@@ -194,6 +194,8 @@ class CodeGenC : public ExprFunctor<void(const PrimExpr&, std::ostream&)>,
   virtual std::string CastFromTo(std::string value, DataType from, DataType target);
   // Get load of single element with expression
   virtual void PrintVecElemLoadExpr(DataType t, int i, const std::string& value, std::ostream& os);
+  // Print restrict keyword for a given Var if applicable
+  virtual void PrintRestrict(const Var& v, std::ostream& os);

Review comment:
       This is the no-aliasing qualifier for opencl kernels. A compiler hint that pointer aliasing does not occur among the restricted pointer arguments.




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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] ZihengJiang merged pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
ZihengJiang merged pull request #7686:
URL: https://github.com/apache/tvm/pull/7686


   


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [tvm] ZihengJiang commented on a change in pull request #7686: [Texture support][Part 1] TIR lowering and OpenCL support

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on a change in pull request #7686:
URL: https://github.com/apache/tvm/pull/7686#discussion_r683025273



##########
File path: src/tir/transforms/lower_tvm_builtin.cc
##########
@@ -98,6 +98,15 @@ class BuiltinLower : public StmtExprMutator {
     }
   }
 
+  Stmt VisitStmt_(const LetStmtNode* op) final {

Review comment:
       Why does mutation happens in `LetStmtNode` instead of `CallNode` directly?

##########
File path: src/target/source/codegen_c.h
##########
@@ -194,6 +194,8 @@ class CodeGenC : public ExprFunctor<void(const PrimExpr&, std::ostream&)>,
   virtual std::string CastFromTo(std::string value, DataType from, DataType target);
   // Get load of single element with expression
   virtual void PrintVecElemLoadExpr(DataType t, int i, const std::string& value, std::ostream& os);
+  // Print restrict keyword for a given Var if applicable
+  virtual void PrintRestrict(const Var& v, std::ostream& os);

Review comment:
       What does `restrict` mean 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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