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/12/13 15:12:59 UTC

[GitHub] [tvm] Lunderberg opened a new pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

Lunderberg opened a new pull request #9727:
URL: https://github.com/apache/tvm/pull/9727


   This is an implementation of [RFC#0039](https://github.com/apache/tvm-rfcs/pull/39), which adds layout transformations to both flat and non-flat memory buffers.
   
   The commits in this PR are split into 4 sub-sections.  Subsection 1 is a refactor that should maintain all existing functionality, while enabling the feature additions in later subsections.  Subsections 2-4 are feature additions, each of which starts with a commit adding unit tests for that feature.
   
   1. Replacing Load/Store nodes with BufferLoad/BufferStore nodes throughout the entire lowering flow.  Access into flat memory regions, previously represented by Load/Store nodes, are now represented by BufferLoad/BufferStore nodes that access a 1-d memory space.
   2. Implement `.transform_layout()`, a scheduling step in TE that alters the in-memory layout of a buffer.
   3. Add `te.AXIS_SEPARATOR` to `.transform_layout()` to define groups of transformed axes, where each group is flattened to a single axis.  These non-flat memory regions are represented by N-d buffer objects, as enabled in step 1.
   4. Expose the transformed axes as the return value of `.transform_layout()`, similar to the existing `.fuse()` and `.split()` functions, to allow additional manipulation by the user.
   
   The commits are organized internally by `git merge --no-ff` commits for each sub-section.  Unfortunately, it looks like those don't show up very cleanly in github's linear list of commits, but can be seen with `git log --graph --format=format:'%h - %s %d'`, the output of which is below.
   
   <details>
   <summary>Output of <code>git log --graph --format=format:'%h - %s %d'</code></summary>
   <pre><code>
   *   e017a87c5 - Breakpoint, expose the transformed axes for use in TE scheduling.  (HEAD -> physical_layout)
   |\  
   | * d89f21479 - [TE] Return transformed iteration variables 
   | * 9de453da0 - [TE] Rewrite loop iteration order 
   | * 69f59e6ca - [TE] Implement te::Transform 
   | * 12f6c2dd0 - [UnitTest] Added tests for loop iteration order. 
   |/  
   *   9868fd5c7 - Breakpoint, axis separators defined. 
   |\  
   | * 1bdb548e3 - [TE] Fill BufferNode::axis_separators from StageNode 
   | * 749a3c651 - [TE] Added Stage::set_axis_separators. 
   | * 3e3c04ebe - [TIR] Added BufferNode::axis_separators 
   | * 3317e1d4c - [UnitTest] Test N-d indices exposed to low-level codegen 
   |/  
   *   0e426b45a - Breakpoint, layout_transform implemented. 
   |\  
   | * 7ce30a2dc - [TIR] Expose tir.transform.ApplyPhysicalLayout for testing 
   | * a78ec6f52 - [TIR] Added ApplyLayoutTransforms as part of StorageFlatten. 
   | * db0c190a9 - [TIR] Added PrimFunc attribute "layout_transform_map", filled from TE. 
   | * 1d0fe1098 - [TE] Added Stage.transform_layout to the Python TE interface. 
   | * 46e173d9f - [TE] Added Stage::transform_layout to the C++ TE implementation. 
   | * 5c278837f - [TIR] Added IndexMap class. 
   | * bc9c5d242 - [UnitTest] Add unit tests to test physical layout remapping. 
   |/  
   *   d2b2a52f5 - Breakpoint, removed Store/Load nodes from use. 
   |\  
   | * 7453a8b22 - Added pre_flattened_shape/pre_flattened_stride fields to Buffer. 
   | * 5bd3fd619 - Replace Store/Load with BufferStore/BufferLoad in ir_builder 
   | * 31698f0d5 - Updated Buffer::vstore/vload to return BufferLoad/BufferStore objects. 
   | * 79c6f1ae0 - Updated tvm::address_of() to hold BufferLoad instead of Load. 
   | * 363d8c507 - Replacing Load/Store in codegens. 
   | * 4687caa4a - Replacing Store/Load in lowering/legalization passes. 
   | * b7daea9c0 - Replacing Store/Load in analysis functions 
   | * 8f8eeb40d - Replacing Store/Load in utility passes. 
   | * 347865de6 - Replacing Store/Load in StorageFlatten 
   | * ffd0737a0 - Removing Store/Load from examples 
   | * be1555d91 - Removing Store/Load from optimization passes 
   | * 9debd1f55 - Replacing Store/Load in Stmt/Expr Visitor/Mutator 
   | * 4916d2c87 - [TIR] Added BufferLoadNode::LegalizeDtype 
   |/  
   * 5557b8c4e - Improve tvmc error message from lazy-loading frontend imports (#9074)  (upstream/main, main)
   </code></pre>
   </details>
   


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

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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/expr.cc
##########
@@ -1056,12 +1058,28 @@ TVM_STATIC_IR_FUNCTOR(ReprPrinter, vtable)
     .set_dispatch<AnyNode>([](const ObjectRef& node, ReprPrinter* p) { p->stream << "?"; });
 
 // BufferLoad
+void BufferLoadNode::LegalizeDtype() {
+  int index_lanes = 1;

Review comment:
       It is over generalization to allow vector indices in all dimensions. Consider restricting the vector indices to only in the last dimension for now `indices[-1]`.

##########
File path: include/tvm/tir/expr.h
##########
@@ -610,6 +610,14 @@ class BufferLoadNode : public PrimExprNode {
   /*! \brief The indices location to be loaded. */
   Array<PrimExpr> indices;
 
+  /*! \brief Set the dtype based on the buffer/indices
+   *
+   * Usually, this will be the same dtype as the buffer.  This may
+   * have a different number of lanes than the buffer's dtype if index
+   * values have more than 1 lane.
+   */
+  void LegalizeDtype();

Review comment:
       DType(to be consistent with other places, as DType is short for DataType). please add comment that this is only intended to be called from ctor and after COW. Consider add an underscore after function to indicate the intended rare behavior. Even better, we might want to only make it private and expose to limited locations via friend, but this can be left as a TODO

##########
File path: src/tir/transforms/storage_rewrite.cc
##########
@@ -1118,6 +1204,11 @@ class VectorTypeAccessChecker : public StmtExprVisitor {
       var_info.element_dtype = value_dtype.element_of();
     }
 
+    int index_lanes = 1;

Review comment:
       restrict vector index to the last dimension 




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/index_map.h
##########
@@ -0,0 +1,137 @@
+/*
+ * 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 tvm/tir/index_map.h
+ * \brief Defines a remapping of buffer indices
+ *
+ * For use with tvm::tir::Buffer.
+ */
+#ifndef TVM_TIR_INDEX_MAP_H_
+#define TVM_TIR_INDEX_MAP_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/array.h>
+#include <tvm/runtime/object.h>
+#include <tvm/tir/var.h>
+
+namespace tvm {
+namespace tir {
+
+/*!
+ * \brief Defines the mapping from logical layout of a tensor to

Review comment:
       Good point, and that comment was leftover from an earlier implementation that restricted a Buffer to a single transform, which represented the logical->physical layout.  I've updated the comment to remove the specificity, since it is now more generally applicable.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/ir_utils.cc
##########
@@ -111,26 +111,56 @@ class IRConvertSSA final : public StmtExprMutator {
       return StmtExprMutator::VisitExpr_(op);
     }
   }
+
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    PrimExpr expr = StmtExprMutator::VisitExpr_(op);
-    op = expr.as<LoadNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Load(op->dtype, scope_[v].back(), op->index, op->predicate);
-    } else {
-      return expr;
-    }
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
+
   Stmt VisitStmt_(const StoreNode* op) final {
-    Stmt stmt = StmtExprMutator::VisitStmt_(op);
-    op = stmt.as<StoreNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Store(scope_[v].back(), op->value, op->index, op->predicate);
-    } else {
-      return stmt;
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(node);

Review comment:
       Thank you.  Updated here, along with all other uses of the same `VisitBufferAccess` pattern.  Calls into `VisitBufferAccess` are made with `std::move(node)` so that the `CopyOnWrite` functionality can avoid copies where possible.  Returns from `VisitBufferAccess` are by value, in order to have copy elision.




-- 
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] jroesch commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/arg_binder.cc
##########
@@ -184,38 +194,41 @@ void ArgBinder::BindDLTensor(const Buffer& buffer, const PrimExpr& device_type,
                                      IntImm(DataType::Int(32), buffer->data_alignment), nop));
   }
 
+  // shape field
+  Buffer buf_shape = decl_buffer({IntImm(DataType::Int(32), pre_flattened_shape.size())},
+                                 tvm_shape_type, arg_name + ".shape");
   Var v_shape(arg_name + ".shape", DataType::Handle());
   def_handle_dtype_.Set(v_shape, make_const(tvm_shape_type, 0));
   init_nest_.emplace_back(
-      LetStmt(v_shape, TVMArrayGet(DataType::Handle(), handle, builtin::kArrShape), nop));
-  for (size_t k = 0; k < buffer->shape.size(); ++k) {
+      LetStmt(buf_shape->data, TVMArrayGet(DataType::Handle(), handle, builtin::kArrShape), nop));
+  for (size_t k = 0; k < pre_flattened_shape.size(); ++k) {
     if (dtype == DataType::Int(4) || dtype == DataType::UInt(4) || dtype == DataType::Int(1)) {
       break;
     }
     std::ostringstream field_name;
     field_name << v_shape->name_hint << '[' << k << ']';
-    Bind_(buffer->shape[k],
-          cast(buffer->shape[k].dtype(),
-               Load(tvm_shape_type, v_shape, IntImm(DataType::Int(32), k), const_true(1))),
-          field_name.str(), true);
+    Bind_(
+        pre_flattened_shape[k],
+        cast(pre_flattened_shape[k].dtype(), BufferLoad(buf_shape, {IntImm(DataType::Int(32), k)})),
+        field_name.str(), true);
   }
   // strides field
-  Var v_strides(arg_name + ".strides", DataType::Handle());
-  def_handle_dtype_.Set(v_strides, tir::TypeAnnotation(tvm_shape_type));
-  init_nest_.emplace_back(
-      LetStmt(v_strides, TVMArrayGet(DataType::Handle(), handle, builtin::kArrStrides), nop));
-  PrimExpr v_strides_is_null = Call(DataType::Bool(1), builtin::isnullptr(), {v_strides});
-  if (buffer->strides.size() == 0) {
+  Buffer buf_strides = decl_buffer({IntImm(DataType::Int(32), pre_flattened_strides.size())},
+                                   tvm_shape_type, arg_name + ".strides");

Review comment:
       Can we put the name formatting behind a helper function to make sure there is only one place we update?




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: python/tvm/tir/ir_builder.py
##########
@@ -442,26 +447,23 @@ def pointer(self, content_type, name="ptr", scope=""):
         ptr : BufferVar
             The buffer var representing the buffer.
         """
-        buffer_var = _expr.Var(name, PointerType(PrimType(content_type), scope))
-        return BufferVar(self, buffer_var, None, content_type)
+        buffer = _buffer.decl_buffer(shape=[], dtype=content_type, name=name, scope=scope)
+        return BufferVar(self, buffer, [], content_type)
 
-    def buffer_ptr(self, buf, shape=None):
+    def buffer_ptr(self, buf):
         """Create pointer variable corresponds to buffer ptr.
 
         Parameters
         ----------
         buf : Buffer
             The buffer to be extracted.
 
-        shape : Tuple
-            Optional shape of the buffer. Overrides existing buffer shape.
-
         Returns
         -------
         ptr : BufferVar
             The buffer var representing the buffer.
         """
-        return BufferVar(self, buf.data, buf.shape if shape is None else shape, buf.dtype)
+        return BufferVar(self, buf, [], buf.dtype)

Review comment:
       Thank you for the catch, and that's something I to refactored out.  On the main branch, that is used to indicate the shape of the buffer, since the data pointer itself doesn't have any shape information.  After the PR, the buffer itself holds the shape information, so the additional array isn't needed.




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening

Review comment:
       To add on @Lunderberg 's options, #7488 added a special memory scope that allows multidimensional memory buffer. Pro: Doesn't add too much complexity to NDArray; Con: It can't support arbitrary `axis_separator`.
   cc @ZihengJiang @masahi 




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/index_map.cc
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 index_map.cc
+ */
+
+#include "tvm/tir/index_map.h"
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/arith/iter_affine_map.h>
+#include <tvm/tir/op.h>
+
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+
+IndexMap::IndexMap(Array<Var> initial_indices, Array<PrimExpr> final_indices) {
+  auto n = make_object<IndexMapNode>();
+  n->initial_indices = std::move(initial_indices);
+  n->final_indices = std::move(final_indices);
+  data_ = std::move(n);
+}
+
+IndexMap IndexMap::Inverse(Array<Range> initial_ranges) const {
+  // Dummy variables to represent the inverse's inputs.
+  Array<Var> output_vars;
+  for (size_t i = 0; i < (*this)->final_indices.size(); i++) {
+    PrimExpr index = (*this)->final_indices[i];
+    // TODO(Lunderberg): Better names for these variables.  A variable
+    // that is passed through unmodified (`index` is an element of
+    // `initial_indices`) should use that input index's name.  A pair
+    // of output indices variables split from a single input index
+    // should be named (X.outer,X.inner).
+    std::stringstream ss;
+    ss << "axis" << i;
+    Var var_index(ss.str(), index.dtype());
+    output_vars.push_back(var_index);
+  }
+
+  // Dummy ranges for the extent of each input.
+  Map<Var, Range> input_iters;
+  ICHECK_EQ((*this)->initial_indices.size(), initial_ranges.size());
+  for (size_t i = 0; i < initial_ranges.size(); i++) {
+    input_iters.Set((*this)->initial_indices[i], initial_ranges[i]);
+  }
+
+  // Unpack the output indices into linear combinations of the initial
+  // indices.
+  arith::Analyzer analyzer;
+  auto iter_map = DetectIterMap((*this)->final_indices, input_iters, 1, true, &analyzer);
+  CHECK(iter_map.size()) << "Index transformation was not bijective.";
+
+  // Determine expressions for the input variables, in terms of the
+  // output variables.
+  Map<Var, PrimExpr> inverse_exprs_map =
+      InverseAffineIterMap(iter_map, Array<PrimExpr>(output_vars.begin(), output_vars.end()));
+
+  // Unpack the map to an array, maintaining the same parameter order.
+  Array<PrimExpr> inverse_exprs;
+  for (const auto& index : (*this)->initial_indices) {
+    inverse_exprs.push_back(inverse_exprs_map.at(index));
+  }
+
+  return IndexMap(output_vars, inverse_exprs);
+}
+
+Array<PrimExpr> IndexMapNode::MapIndices(const Array<PrimExpr>& indices) const {
+  ICHECK_EQ(indices.size(), initial_indices.size());
+
+  arith::Analyzer analyzer;
+
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    analyzer.Bind(initial_indices[i], indices[i]);
+  }
+
+  Array<PrimExpr> output;
+  for (const auto& output_dim : final_indices) {
+    output.push_back(analyzer.Simplify(output_dim));
+  }
+
+  return output;
+}
+
+Array<Range> IndexMapNode::MapRanges(const Array<Range>& ranges) const {
+  ICHECK_EQ(ranges.size(), initial_indices.size());
+
+  Map<Var, Range> input_iters;
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    input_iters.Set(initial_indices[i], ranges[i]);
+  }
+
+  arith::Analyzer analyzer;
+  auto iter_sums = DetectIterMap(final_indices, input_iters, 1, true, &analyzer);
+
+  Array<Range> output;
+  for (const auto& iter_sum : iter_sums) {
+    PrimExpr min = iter_sum->base;
+    PrimExpr extent = 0;
+    for (const auto& term : iter_sum->args) {
+      extent += term->extent * term->scale;
+    }
+    output.push_back(Range::FromMinExtent(min, extent));
+  }
+
+  return output;
+}
+
+Array<PrimExpr> IndexMapNode::MapShape(const Array<PrimExpr>& shape) const {
+  ICHECK_EQ(shape.size(), initial_indices.size());
+
+  Array<Range> ranges;
+  for (auto& dim : shape) {
+    ranges.push_back(Range(0, dim));
+  }
+  Array<Range> mapped = MapRanges(std::move(ranges));
+
+  Array<PrimExpr> output;
+  for (auto& range : mapped) {
+    ICHECK(is_zero(range->min));
+    output.push_back(range->extent);
+  }
+

Review comment:
       At present all mappings are surjective, but I'd like to add in the future.  The use case would be to have implied padding be present in the physical layout, with elements that don't have a representation in the logical layout.  For example, if a 1-d tensor has size 50 and warp size 32, calling `.transform_layout(lambda i: [i//warp_size, i%warp_size])` could map it to the [2,32] shape, even though that increases the total number of elements.
   
   That said, I haven't fully thought through the implications of that, and some of the use cases would require a specific value for the padding (e.g. padding a tensor out with 0 prior to running conv2d with pad_value=0).




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/ir_utils.cc
##########
@@ -111,26 +111,56 @@ class IRConvertSSA final : public StmtExprMutator {
       return StmtExprMutator::VisitExpr_(op);
     }
   }
+
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    PrimExpr expr = StmtExprMutator::VisitExpr_(op);
-    op = expr.as<LoadNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Load(op->dtype, scope_[v].back(), op->index, op->predicate);
-    } else {
-      return expr;
-    }
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
+
   Stmt VisitStmt_(const StoreNode* op) final {
-    Stmt stmt = StmtExprMutator::VisitStmt_(op);
-    op = stmt.as<StoreNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Store(scope_[v].back(), op->value, op->index, op->predicate);
-    } else {
-      return stmt;
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(node);
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto node = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return VisitBufferAccess(node);

Review comment:
       Thank you, updated here as well.




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

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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/builtin.h
##########
@@ -105,10 +105,10 @@ TVM_DLL const Op& large_uint_imm();
 TVM_DLL const Op& q_multiply_shift();
 
 /*!
- * \brief See pesudo code
+ * \brief See pseudo code
  *
- *  Handle address_of(Load *op) {
- *     return &op->buffer_var[index];

Review comment:
       Consider restricting this to the case when len(op->indices) == 1 for now, if we do not need other cases




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/buffer.cc
##########
@@ -407,22 +515,31 @@ PrimExpr Buffer::access_ptr(int access_mask, DataType ptr_type, int content_lane
 
 Buffer::Buffer(Var data, DataType dtype, Array<PrimExpr> shape, Array<PrimExpr> strides,
                PrimExpr elem_offset, String name, int data_alignment, int offset_factor,
-               BufferType buffer_type, Span span) {
+               BufferType buffer_type, Array<IntImm> axis_separators, Span span) {
   DataType storage_dtype = dtype;
   // specially handle bool
   if (storage_dtype == DataType::Bool()) {
     storage_dtype = DataType::Int(8);
   }
-  ICHECK(IsPointerType(data->type_annotation, storage_dtype))
-      << "Buffer data field expect to have the right pointer type annotation"
-      << " annotation=" << data->type_annotation << ", storage_dtype=" << storage_dtype;
+  // The buffer dtype may differ from the dtype of the underlying

Review comment:
       Agreed, explicit casts will be better, and should be included in later updates.  (Maybe also handling bool with explicit casts as well, to minimize the special case code for it.)  Added a TODO to make sure that it gets revisited.




-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/buffer.cc
##########
@@ -407,22 +515,31 @@ PrimExpr Buffer::access_ptr(int access_mask, DataType ptr_type, int content_lane
 
 Buffer::Buffer(Var data, DataType dtype, Array<PrimExpr> shape, Array<PrimExpr> strides,
                PrimExpr elem_offset, String name, int data_alignment, int offset_factor,
-               BufferType buffer_type, Span span) {
+               BufferType buffer_type, Array<IntImm> axis_separators, Span span) {
   DataType storage_dtype = dtype;
   // specially handle bool
   if (storage_dtype == DataType::Bool()) {
     storage_dtype = DataType::Int(8);
   }
-  ICHECK(IsPointerType(data->type_annotation, storage_dtype))
-      << "Buffer data field expect to have the right pointer type annotation"
-      << " annotation=" << data->type_annotation << ", storage_dtype=" << storage_dtype;
+  // The buffer dtype may differ from the dtype of the underlying

Review comment:
       The storage type consistency is an assumption that was being made before. Not sure if such generalization is OK. 
   
   Ideally the common type sharing should be done with an explicit cast of pointer type, rather than implicit cast here.
   
   Would be great to leave a TODO to revisit here, cc @Lunderberg @vinx13 @junrushao1994 




-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/printer/tvmscript_printer.cc
##########
@@ -1008,8 +1013,59 @@ Doc TVMScriptPrinter::VisitStmt_(const BufferRealizeNode* op) {
   return Doc();
 }
 
+namespace {
+struct AllocUsage {
+  Buffer alloc_buffer;
+  Array<Buffer> aliasing_buffers;
+};
+
+template <typename AllocNode>
+AllocUsage find_allocate_usage(AllocNode* op, Map<Var, Array<Buffer>>* cache_ptr) {

Review comment:
       nit: CamelCase




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/function.h
##########
@@ -91,11 +91,23 @@ class PrimFuncNode : public BaseFuncNode {
    */
   Map<tir::Var, Buffer> buffer_map;
 
+  /*! \brief The buffer map prior to flattening.
+   *
+   * This contains the buffers as they exists prior to flattening, and
+   * is used for validating an input tensor passed into the packed
+   * API.  Any buffer that is present in `buffer_map` but not present
+   * in `preflattened_buffer_map` is assumed to be the same before
+   * and after flattening (e.g. a 1-d tensor that is backed by 1-d
+   * flat memory).
+   */
+  Map<tir::Var, Buffer> preflattened_buffer_map;

Review comment:
       That's correct, it's for the validation of user-provided tensors.  The buffer_map defines the buffers that will be used in the body of the function, and is updated with the flattened buffer definitions either in `StorageFlatten` for TE-based schedules or `FlattenBuffer` for TIR-based schedules.  However, the unpacking of arguments and insertion of checks to validate  user-provided input isn't added until much later, during the `MakePackedAPI` pass.  Unless the buffer's shape prior to flattening is available, the check would be generated based on the post-flattening shape, which would be a major user-facing change.  (e.g. Needing to pass a numpy array of shape `[6]` to match the flattened shape of a `[2,3]` tensor, rather than the original `[2,3]` shape.)
   
   That said, I'd like to it in future PRs, by keeping the buffer_map untouched, but defining a flattened buffer that aliases the original buffer.  This will remove the need for the buffer_map, but would make it trickier for passes to manage which buffer objects have replaced which other buffer objects, and what relationships need to be maintained.  It's not hard to do for any one pass, but is tedious, and so I want to first improve the ergonomics of `StmtExprMutator`'s handling of buffers before removing the preflattened_buffer_map.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       I've changed my mind, and agree that keeping a reference to the pre-flattened Buffer would be much cleaner.  I needed to add a third similar parameter for the preflattened type, since the post-flattening type of a boolean array is int8, and three similar parameters is too much.
   
   Unfortunately, having a `Optional<Buffer> pre_flattened_buffer` parameter causes some compilation issues.  `BufferNode` cannot be declared before `Buffer`, because `Optional<Buffer>` cannot use a forward declaration.   `Buffer` cannot be declared before `BufferNode`, because `TVM_DEFINE_OBJECT_REF_METHODS` nor the `TVM_DEFINE_OBJECT_REF_COW_METHODS` macros work with a forward declaration.  It is possible to copy-paste declare the methods defined by these macros, then move the definitions themselves into `buffer.cc`, but that has more copy-paste code than I'd like.  I have a rough draft of the changes required, but I think it should be a follow-up PR 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.

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

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



[GitHub] [tvm] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Rebasing onto main to resolve merge conflicts.  Apologies in advance for any in-progress reviews that get orphaned by the rebase.


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: python/tvm/tir/ir_builder.py
##########
@@ -442,26 +447,23 @@ def pointer(self, content_type, name="ptr", scope=""):
         ptr : BufferVar
             The buffer var representing the buffer.
         """
-        buffer_var = _expr.Var(name, PointerType(PrimType(content_type), scope))
-        return BufferVar(self, buffer_var, None, content_type)
+        buffer = _buffer.decl_buffer(shape=[], dtype=content_type, name=name, scope=scope)
+        return BufferVar(self, buffer, [], content_type)
 
-    def buffer_ptr(self, buf, shape=None):
+    def buffer_ptr(self, buf):
         """Create pointer variable corresponds to buffer ptr.
 
         Parameters
         ----------
         buf : Buffer
             The buffer to be extracted.
 
-        shape : Tuple
-            Optional shape of the buffer. Overrides existing buffer shape.
-
         Returns
         -------
         ptr : BufferVar
             The buffer var representing the buffer.
         """
-        return BufferVar(self, buf.data, buf.shape if shape is None else shape, buf.dtype)
+        return BufferVar(self, buf, [], buf.dtype)

Review comment:
       Thank you for the catch, and that's something that can be removed.  On the main branch, that is used to indicate the shape of the buffer, since the data pointer itself doesn't have any shape information.  After the PR, the buffer itself holds the shape information, so the additional array isn't needed.




-- 
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] jroesch commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: python/tvm/tir/ir_builder.py
##########
@@ -442,26 +447,23 @@ def pointer(self, content_type, name="ptr", scope=""):
         ptr : BufferVar
             The buffer var representing the buffer.
         """
-        buffer_var = _expr.Var(name, PointerType(PrimType(content_type), scope))
-        return BufferVar(self, buffer_var, None, content_type)
+        buffer = _buffer.decl_buffer(shape=[], dtype=content_type, name=name, scope=scope)
+        return BufferVar(self, buffer, [], content_type)
 
-    def buffer_ptr(self, buf, shape=None):
+    def buffer_ptr(self, buf):
         """Create pointer variable corresponds to buffer ptr.
 
         Parameters
         ----------
         buf : Buffer
             The buffer to be extracted.
 
-        shape : Tuple
-            Optional shape of the buffer. Overrides existing buffer shape.
-
         Returns
         -------
         ptr : BufferVar
             The buffer var representing the buffer.
         """
-        return BufferVar(self, buf.data, buf.shape if shape is None else shape, buf.dtype)
+        return BufferVar(self, buf, [], buf.dtype)

Review comment:
       Why is this empty list 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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/lower_custom_datatypes.cc
##########
@@ -103,32 +103,59 @@ class CustomDatatypesLowerer : public StmtExprMutator {
     }
   }
 
-  PrimExpr VisitExpr_(const LoadNode* load) final {
-    bool to_be_lowered = datatype::Registry::Global()->GetTypeRegistered(load->dtype.code());
-    PrimExpr expr = StmtExprMutator::VisitExpr_(load);
-    load = expr.as<LoadNode>();
-    if (to_be_lowered) {
-      auto new_load_type = DataType::UInt(load->dtype.bits());
-      auto buffer_var = load->buffer_var;
-      auto it = var_remap_.find(buffer_var);
-      if (it != var_remap_.end()) {
-        buffer_var = it->second;
-      }
-      return Load(new_load_type, buffer_var, load->index, load->predicate);
-    }
-    return expr;
+  PrimExpr VisitExpr_(const LoadNode* op) final {
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";

Review comment:
       Not strictly needed, but I kept them in to show that their exclusion was intentional, without needing to look elsewhere in the codebase.  Otherwise, somebody might see that a particular pass doesn't handle `LoadNode*` and unnecessarily add it back in.  Since the LoadNode constructor also throws an error, I'm also okay with removing these visitors, as any use would be caught by unit tests.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       I've changed my mind, and agree that keeping a reference to the pre-flattened Buffer would be much cleaner.  I needed to add a third similar parameter for the preflattened type, since the post-flattening type of a boolean array is int8, and three similar parameters is too much.
   
   Unfortunately, having a `Optional<Buffer> pre_flattened_buffer` parameter causes some compilation issues.  `BufferNode` cannot be declared before `Buffer`, because `Optional<Buffer>` cannot use a forward declaration.   `Buffer` cannot be declared before `BufferNode`, because `TVM_DEFINE_OBJECT_REF_METHODS` nor the `TVM_DEFINE_OBJECT_REF_COW_METHODS` macros work with a forward declaration.  It is possible to copy-paste declare the methods defined by these macros, then move the definitions themselves into `buffer.cc`, but that has more copy-paste code than I'd like.  I have a rough draft of the changes required, but I think it should be a follow-up PR 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.

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

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



[GitHub] [tvm] areusch commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   cc @Mousius 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.

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

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



[GitHub] [tvm] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/target/source/codegen_c.cc
##########
@@ -158,78 +158,52 @@ void CodeGenC::PrintSSAAssign(const std::string& target, const std::string& src,
 }
 
 // Print a reference expression to a buffer.
-std::string CodeGenC::GetBufferRef(DataType t, const VarNode* buffer, PrimExpr index) {
+std::string CodeGenC::GetBufferRef(DataType t, const BufferNode* buffer, PrimExpr index) {
+  const VarNode* buffer_var = buffer->data.get();
   std::ostringstream os;
-  std::string vid = GetVarID(buffer);
+  std::string vid = GetVarID(buffer_var);
   std::string scope;
-  if (alloc_storage_scope_.count(buffer)) {
-    scope = alloc_storage_scope_.at(buffer);
+  if (alloc_storage_scope_.count(buffer_var)) {
+    scope = alloc_storage_scope_.at(buffer_var);
   }
-  bool is_vol = IsVolatile(buffer);
-  if (t.lanes() == 1) {
-    if (!HandleTypeMatch(buffer, t) || is_vol) {
-      os << "((";
-      if (is_vol) {
-        os << "volatile ";
-      }
-      // Scope may not be part of type.
-      if (!scope.empty() && IsScopePartOfType()) {
-        PrintStorageScope(scope, os);
-      }
-      PrintType(t, os);
-      os << "*)" << vid << ')';
-    } else {
-      os << vid;
-    }
-    os << "[(";
-    PrintExpr(index, os);
-    os << ")";
-    if (t.bits() == 4 || (t.bits() == 1 && t.is_int())) {
-      os << " / " << (32 / t.bits());
-    }
-    os << ']';
-  } else {

Review comment:
       does this case go away with the new vectorization impl?

##########
File path: src/tir/transforms/vectorize_loop.cc
##########
@@ -62,30 +62,89 @@ class VecAllocAccess : public StmtExprMutator {
  public:
   VecAllocAccess(const VarNode* buf, Var var, int var_lanes)
       : buf_(buf), var_(var), var_lanes_(var_lanes) {}
-  // Load
+
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    PrimExpr expr = StmtExprMutator::VisitExpr_(op);
-    op = expr.as<LoadNode>();
-    if (op->buffer_var.get() == buf_) {
-      return Load(op->dtype, op->buffer_var, op->index * var_lanes_ + var_, op->predicate);
-    } else {
-      return expr;
-    }
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
-  // Store
+
   Stmt VisitStmt_(const StoreNode* op) final {
-    Stmt stmt = StmtExprMutator::VisitStmt_(op);
-    op = stmt.as<StoreNode>();
-    if (op->buffer_var.get() == buf_) {
-      return Store(op->buffer_var, op->value, op->index * var_lanes_ + var_, op->predicate);
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto load = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return UpdateBufferAccess(load);
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto store = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return UpdateBufferAccess(store);
+  }
+
+ private:
+  template <typename Node>
+  Node UpdateBufferAccess(Node node) {
+    // Only update the buffer that's being replaced.
+    if (node->buffer->data.get() != buf_) {
+      return node;
+    }
+
+    arith::Analyzer analyzer;

Review comment:
       nit: make it a class member

##########
File path: tests/python/unittest/test_tir_transform_lower_cross_thread_reduction.py
##########
@@ -25,7 +25,14 @@
 def _check(original, transformed):
     mod = tvm.IRModule.from_expr(original)
     mod = tvm.tir.transform.LowerCrossThreadReduction()(mod)
-    tvm.ir.assert_structural_equal(mod["main"], transformed, True)
+    try:
+        tvm.ir.assert_structural_equal(mod["main"], transformed, True)
+    except ValueError:
+        with open("temp_expected.txt", "w") as f:
+            f.write(transformed.script())
+        with open("temp_observed.txt", "w") as f:
+            f.write(mod["main"].script())

Review comment:
       remove 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] tqchen commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/expr.cc
##########
@@ -1056,12 +1058,28 @@ TVM_STATIC_IR_FUNCTOR(ReprPrinter, vtable)
     .set_dispatch<AnyNode>([](const ObjectRef& node, ReprPrinter* p) { p->stream << "?"; });
 
 // BufferLoad
+void BufferLoadNode::LegalizeDtype() {
+  int index_lanes = 1;

Review comment:
       I agree that could be an useful generalization. 
   
   The main rational of the suggestion is that we might want to start small(that matches the original behavior) then generally analyze the effect and generalize to cases(like at most one vector indices) after thinking about all the consequences. This is mainly to avoid possibly bugs on unsupported cases that we do not yet anticipated
   
   Assuming right now we still restrict vectorize to happen after flattening, the last dimension restriction could serve the current purose, then we can discuss for more generlaization.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -88,6 +128,8 @@ class BufferNode : public Object {
     v->Visit("data", &data);
     v->Visit("dtype", &dtype);
     v->Visit("shape", &shape);
+    v->Visit("pre_flattened_shape", &pre_flattened_shape);
+    v->Visit("pre_flattened_strides", &pre_flattened_strides);

Review comment:
       Thank you for the catch, and fixed.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/target/llvm/codegen_llvm.cc
##########
@@ -931,15 +931,15 @@ llvm::Value* CodeGenLLVM::CreateIntrinsic(const CallNode* op) {
   } else if (op->op.same_as(builtin::tvm_storage_sync())) {
     return CreateStorageSync(op);
   } else if (op->op.same_as(builtin::address_of())) {
-    const LoadNode* l = op->args[0].as<LoadNode>();
-    ICHECK(op->args.size() == 1 && l);
-    TypedPointer buffer_ptr;
-    if (const RampNode* r = l->index.as<RampNode>()) {
-      PrimExpr index = r->base / make_const(DataType::Int(32), r->lanes);
-      buffer_ptr = CreateBufferPtr(l->dtype, MakeValue(l->buffer_var), MakeValue(index));
-    } else {
-      buffer_ptr = CreateBufferPtr(l->dtype, MakeValue(l->buffer_var), MakeValue(l->index));
+    const BufferLoadNode* load = op->args[0].as<BufferLoadNode>();
+    ICHECK(op->args.size() == 1 && load);
+    ICHECK_EQ(load->indices.size(), 0) << "LLVM only supports flat memory allocations.";
+    PrimExpr index = load->indices[0];

Review comment:
       I think `index` needs to be `PrimExpr` here, since conditionally changing it to a `RampNode` in the next few lines wouldn't be possible with a `const PrimExpr&`.  The alternative would be to have everything through the call to `CreateBufferPtr` be duplicated on both sides of a conditional (as below), but I felt that the decrease in readability wouldn't be worth avoiding the copy.
   
   ```c++
   const PrimExpr& index = load->indices[0];
   TypedPointer buffer_ptr;
   if (const RampNode* r = index.as<RampNode>()) {
     PrimExpr ramp_index = r->base / make_const(DataType::Int(32), r->lanes);
     CreateBufferPtr(load->dtype, MakeValue(load->buffer->data), MakeValue(ramp_index));
   } else {
     CreateBufferPtr(load->dtype, MakeValue(load->buffer->data), MakeValue(index));
   }
   ```




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening

Review comment:
       To add on @Lunderberg 's options, #7488 added a special memory scope that allows multidimensional memory buffer. Pro: Doesn't require API change from NDArray; Con: It can't support arbitrary `axis_separator`.
   cc @ZihengJiang @masahi 




-- 
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] tmoreau89 commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   CC - @jroesch @junrushao1994 @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.

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

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



[GitHub] [tvm] Lunderberg edited a comment on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

Posted by GitBox <gi...@apache.org>.
Lunderberg edited a comment on pull request #9727:
URL: https://github.com/apache/tvm/pull/9727#issuecomment-1043004704


   Updated to resolve breakage in vta unit tests.


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -98,10 +113,17 @@ class BufferNode : public Object {
   }
 
   bool SEqualReduce(const BufferNode* other, SEqualReducer equal) const {
-    // Use DefEqual as buffer can define variables
-    // in its semantics, skip name as name is not important.
+    // Use DefEqual as buffer can define variables in its semantics,
+    // skip name as name is not important.
+
+    // The pre-flattened information is only used for type-checking,
+    // and doesn't represent a different computation.
+    //
+    // TODO(Lunderberg): Move the pre-flattened buffer information
+    // into the PrimFunc's buffer_map.

Review comment:
       Thank you, and updated.




-- 
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] jroesch commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   cc @manupa-arm would be great if we Eric could get some review from ARM folks on these changes 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.

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

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



[GitHub] [tvm] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/te/schedule.h
##########
@@ -771,6 +835,36 @@ class Singleton : public IterVarRelation {
   TVM_DEFINE_OBJECT_REF_METHODS(Singleton, IterVarRelation, SingletonNode);
 };
 
+/*!
+ * \brief Transform iterator according to some arbitrary expression.
+ */
+class TransformNode : public IterVarRelationNode {
+ public:
+  Array<IterVar> original_variables;

Review comment:
       Thank you, and added.




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: tests/python/unittest/test_tir_transform_instrument_bound_checkers.py
##########
@@ -94,7 +94,7 @@ def test_out_of_bounds_vectorize_llvm(nn, index_a, index_b):
 @tvm.testing.requires_llvm
 def test_in_bounds_vectorize_llvm():
     n = 512
-    lanes = 2
+    lanes = 1

Review comment:
       is this change needed to pass the test?

##########
File path: include/tvm/tir/buffer.h
##########
@@ -98,10 +113,17 @@ class BufferNode : public Object {
   }
 
   bool SEqualReduce(const BufferNode* other, SEqualReducer equal) const {
-    // Use DefEqual as buffer can define variables
-    // in its semantics, skip name as name is not important.
+    // Use DefEqual as buffer can define variables in its semantics,
+    // skip name as name is not important.
+
+    // The pre-flattened information is only used for type-checking,
+    // and doesn't represent a different computation.
+    //
+    // TODO(Lunderberg): Move the pre-flattened buffer information
+    // into the PrimFunc's buffer_map.

Review comment:
       update the comment as pre-flattened information is no longer here

##########
File path: src/tir/transforms/merge_dynamic_shared_memory_allocations.cc
##########
@@ -294,22 +308,61 @@ class DynamicSharedMemoryRewriter : public StmtExprMutator {
   }
 
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    if (IsDynamicSharedMemory(op->buffer_var)) {
-      PrimExpr offset = GetBufferOffset(op->buffer_var, op->dtype);
-      PrimExpr index = StmtExprMutator::VisitExpr(op->index);
-      return Load(op->dtype, merged_buf_var_, offset + index, op->predicate, op->span);
-    }
-    return StmtExprMutator::VisitExpr_(op);
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
 
   Stmt VisitStmt_(const StoreNode* op) final {
-    if (IsDynamicSharedMemory(op->buffer_var)) {
-      PrimExpr offset = GetBufferOffset(op->buffer_var, op->value->dtype);
-      PrimExpr index = StmtExprMutator::VisitExpr(op->index);
-      PrimExpr value = StmtExprMutator::VisitExpr(op->value);
-      return Store(merged_buf_var_, value, offset + index, op->predicate, op->span);
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(std::move(node));
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto node = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return VisitBufferAccess(std::move(node));
+  }
+
+  template <typename Node>
+  Node VisitBufferAccess(Node node) {
+    if (IsDynamicSharedMemory(node->buffer->data)) {
+      ICHECK_EQ(node->indices.size(), 1)
+          << "MergeDynamicSharedMemoryAllocations expects flat memory buffers, "
+          << "and is to be run after "
+          << "StorageFlatten (TE schedules) or FlattenBuffer (TIR schedules)";
+      Array<PrimExpr> indices = {node->indices[0] +
+                                 this->GetBufferOffset(node->buffer->data, node->buffer->dtype)};
+
+      auto writer = node.CopyOnWrite();
+      writer->buffer = GetUpdatedBuffer(node->buffer);
+      writer->indices = indices;
     }
-    return StmtExprMutator::VisitStmt_(op);
+
+    return node;
+  }
+
+  Buffer GetUpdatedBuffer(Buffer buffer) {
+    auto key = buffer.get();
+    auto it = buffer_remap_.find(key);
+    if (it != buffer_remap_.end()) {
+      return it->second;
+    }
+
+    if (IsDynamicSharedMemory(buffer->data)) {
+      ICHECK_EQ(buffer->shape.size(), 1)
+          << "MergeDynamicSharedMemoryAllocations expects flat memory buffers, "
+          << "and is to be run after "
+          << "StorageFlatten (TE schedules) or FlattenBuffer (TIR schedules)";
+      auto writer = buffer.CopyOnWrite();
+      writer->data = merged_buf_var_;

Review comment:
       This pass consolidates multiple dynamic shared memory buffers into one, shall we also create a buffer `merged_buf_` instead of creating an alias here? 

##########
File path: python/tvm/script/tir/special_stmt.py
##########
@@ -132,6 +132,7 @@ def match_buffer(
             align=-1,
             offset_factor=0,
             buffer_type="default",
+            flatten_buffer=False,

Review comment:
       Update the comment here https://github.com/apache/tvm/blob/e40414fc81d99c235f857ae9c741a4f25d072f79/python/tvm/script/tir/special_stmt.py#L103-L104 and mention the usage of `flatten_buffer`

##########
File path: src/tir/transforms/split_host_device.cc
##########
@@ -155,10 +160,27 @@ class VarUseDefAnalysis : public StmtExprMutator {
   }
 
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    this->HandleUse(op->buffer_var);
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    this->HandleUse(op->buffer->data);
     return StmtExprMutator::VisitExpr_(op);
   }
 
+  void VisitBuffer(Buffer buffer) {
+    this->HandleUse(buffer->data);
+    auto visit_arr = [&](Array<PrimExpr> arr) {
+      for (const auto& element : arr) {
+        this->VisitExpr(element);
+      }
+    };
+
+    visit_arr(buffer->shape);
+    visit_arr(buffer->strides);
+  }
+

Review comment:
       is this used?

##########
File path: include/tvm/tir/function.h
##########
@@ -136,15 +157,31 @@ class PrimFunc : public BaseFunc {
  public:
   /*!
    * \brief Constructor
+   *
    * \param params The parameters of the function.
+   *
    * \param body The body of the function.
+   *
    * \param ret_type The return type of the function.
+   *
    * \param buffer_map The buffer map for parameter buffer unpacking.
+   * This contains buffer objects as they appear in the body of the
+   * PrimFunc.  (e.g. a buffer of shape ``[1024]`` originally
+   * generated as a tensor of shape ``[32, 32]``)
+   *
+   * \param preflattened_buffer_map The buffer map for
+   * parameter buffer unpacking.  This contains buffer
+   * objects as they are expected to be passed in by the
+   * callee.  (e.g. a buffer of shape ``[32, 32]`` originally
+   * generated as a tensor of shape ``[32, 32]``)
+   *
    * \param attrs Additional function attributes.
+   *
    * \param span The location of this object in the source code.
    */
   TVM_DLL PrimFunc(Array<tir::Var> params, Stmt body, Type ret_type = VoidType(),
                    Map<tir::Var, Buffer> buffer_map = Map<tir::Var, Buffer>(),
+                   Map<tir::Var, Buffer> preflattened_buffer_map = Map<tir::Var, Buffer>(),

Review comment:
       use `Optional<Map<tir::Var, Buffer>>` to explicitly mark it as optional




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: tests/python/unittest/test_tir_transform_instrument_bound_checkers.py
##########
@@ -94,7 +94,7 @@ def test_out_of_bounds_vectorize_llvm(nn, index_a, index_b):
 @tvm.testing.requires_llvm
 def test_in_bounds_vectorize_llvm():
     n = 512
-    lanes = 2
+    lanes = 1

Review comment:
       Whoops, that managed to stay in from testing.  I've changed it back to `lanes=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.

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

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



[GitHub] [tvm] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/analysis/device_constraint_utils.cc
##########
@@ -223,6 +225,23 @@ class ApplyDeviceConstraintsMutator : public StmtExprMutator {
         any_change = true;
       }
       new_buffer_map.Set(param, new_buffer);
+
+      // Rewrite the pre-flattened buffers to account for constraint.
+      // This only has an impact if the IRModule being analyzed has
+      // already been run through the StorageFlatten or FlattenBuffer
+      // passes.
+      if (auto opt = prim_func->preflattened_buffer_map.Get(param)) {
+        Buffer pf_buffer = opt.value();
+        if (pf_buffer.same_as(buffer)) {

Review comment:
       Does this mean the preflattened buffer of a buffer is the same as the buffer itself? If so, in what case does this hold?




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/builtin.h
##########
@@ -105,10 +105,10 @@ TVM_DLL const Op& large_uint_imm();
 TVM_DLL const Op& q_multiply_shift();
 
 /*!
- * \brief See pesudo code
+ * \brief See pseudo code
  *
- *  Handle address_of(Load *op) {
- *     return &op->buffer_var[index];

Review comment:
       I think it needs to allow non-flat indices, both for TIR prior to flattening (N-d indices for N-d logical shape of buffer) and for TIR post-flattening on supported targets (N-d indices for N-d physical shape).  I've added to the docstring that the buffer address must be compatible with the targets used.




-- 
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] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Updated to resolve failures in vta unit tests.


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/index_map.h
##########
@@ -0,0 +1,137 @@
+/*
+ * 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 tvm/tir/index_map.h
+ * \brief Defines a remapping of buffer indices
+ *
+ * For use with tvm::tir::Buffer.
+ */
+#ifndef TVM_TIR_INDEX_MAP_H_
+#define TVM_TIR_INDEX_MAP_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/array.h>
+#include <tvm/runtime/object.h>
+#include <tvm/tir/var.h>
+
+namespace tvm {
+namespace tir {
+
+/*!
+ * \brief Defines the mapping from logical layout of a tensor to
+ * physical layout of a buffer.
+ */
+class IndexMapNode : public Object {

Review comment:
       This could go into the `Buffer` along-side the pre-flattened shapes/strides, rather than its current location as a `PrimFunc` attribute, but wouldn't entirely replace them.  The `IndexMap` represents transformations that occur during the [ApplyLayoutTransforms](https://github.com/apache/tvm/blob/6e9e4e6b0/src/tir/transforms/storage_flatten.cc#L1079) pass (TE schedules), or at the time when applied by a user (schedulable TIR).
   
   The constraints that I ran into were as follows:
   
   1. Buffer flattening occurs during `tir.lower`, and is a prerequisite to several other passes.
   2. For a given pre-flattened shape, there exist more than one way to flatten it.
   3. The checks in defined [MakePackedAPI](https://github.com/apache/tvm/blob/6e9e4e6b0/src/driver/driver_api.cc#L627), during [the call](https://github.com/apache/tvm/blob/6e9e4e6b0/src/tir/transforms/make_packed_api.cc#L247) to [ArgBinder::BindDLTensor](https://github.com/apache/tvm/blob/6e9e4e6b0/src/tir/transforms/arg_binder.cc#L149), are on the pre-flattened shape/strides.
   4. The low-level TIR should have direct access to the flattened buffer shape, and flattened indices.
   
   Any one of those constraints, if broken, would avoid the need for the pre-flattened shape/strides.  However, each one came with tradeoffs that I wanted to avoid.
   
   1. If buffer flattening occurs after `MakePackedAPI` call, then it wouldn't need separate storage, because the pre-flattened shape/strides would still be present as the the buffer's shape/strides.  However, this would require significant changes to all passes in-between.
   2. If there were only a single way to flatten a buffer, then the Buffer could always store the pre-flattened shape/strides, and the flattened shape could be generated as needed.  This was how the previous version of the code worked, since the only flattening was flattening to a single flat memory buffer.
   3. If the checks were based on the post-flattened shape/strides, then MakePackedAPI wouldn't need the pre-flattened shape/strides.  However, since these are checked against the array shape passed by the user, that would be a significant breaking change.
   4. If the buffer maintained the pre-flattened shape/strides, and generated the post-flattened shape as needed, then the buffer could maintain the pre-flattened shape through to the low-level TIR.  However, this would require passes to know whether they are before/after flattening, and introduce different semantics based on that location.  (e.g. Before flattening, `buffer_load->indices.size() == buffer_load->buffer.shape.size()`.  After flattening, `buffer_load->indices.size() == buffer_load->buffer.GetFlattenedBuffer().shape.size()`.)  I didn't want to introduce that dependency on location in the lowering flow.




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

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

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



[GitHub] [tvm] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   @vinx13 @csullivan @tmoreau89 @adstraw @cconvey 


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -127,7 +174,7 @@ class BufferNode : public Object {
    * without adjusting for number of lanes.  (e.g. The number of
    * float16x4 elements in a buffer of type float16x4.)
    */
-  PrimExpr ElemOffset(Array<PrimExpr> index) const;
+  Array<PrimExpr> ElemOffset(Array<PrimExpr> index) const;

Review comment:
       Good point, though I think there would need to be some additional terminology introduced.  The options I had thought about were below.
   
   * Keep the same function names.  Not as informative as it could be, because it doesn't indicate whether the input/output indices correspond to specific indices elsewhere in the flow (e.g. those specified by the user or expressed to low-level code generation).
   * Rename to `Array<PrimExpr> GetPhysicalIndex(Array<PrimExpr> logical_index)`.  Not quite true, since the indices passed in should be after any layout transformations have been applied, but before flattening.  `logical_index` would imply that it takes indices exactly as specified in the `te.compute` definition.
   * Rename to `Array<PrimExpr> GetPhysicalIndex(Array<PrimExpr> transformed_index)`.  Not quite true, since it implies that passing in post-transformation indices can always output the physical index.  The Buffer object must be modified by the transformation first, before the transformed index can be passed in.
   * Rename to `Array<PrimExpr> FlattenIndex(Array<PrimExpr> index)`.  Actually, now that I look at that, I kind of like that one.  It doesn't imply additional functionality, and 
   
   After writing out the different options, I think it makes sense to rename it to `FlattenIndex`, along with adding a comment that this should be used after all transformations have been applied, and produces the physical index.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/target/llvm/codegen_llvm.cc
##########
@@ -1189,14 +1189,24 @@ llvm::Value* CodeGenLLVM::VisitExpr_(const LetNode* op) {
 }
 
 llvm::Value* CodeGenLLVM::VisitExpr_(const LoadNode* op) {
+  LOG(FATAL) << "Unexpected deprecated LoadNode.  Use BufferLoadNode instead.";
+  return NULL;
+}
+
+llvm::Value* CodeGenLLVM::VisitExpr_(const BufferLoadNode* op) {
+  ICHECK_EQ(op->indices.size(), 1) << "CodeGenLLVM expects flattened 1-d buffers.";
+
   DataType t = op->dtype;
-  bool is_volatile = volatile_buf_.count(op->buffer_var.get());
-  llvm::Value* buffer = MakeValue(op->buffer_var);
-  llvm::Value* index = MakeValue(op->index);
+  Var buffer_var = op->buffer->data;
+  PrimExpr buffer_index = op->indices[0];

Review comment:
       Thank you, and fixed.




-- 
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] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Stack of commits were to rebase these changes onto main, since some of the TIR unittests require functionality introduced in #10099.


-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/expr.h
##########
@@ -638,6 +630,22 @@ class BufferLoadNode : public PrimExprNode {
 
   static constexpr const char* _type_key = "tir.BufferLoad";
   TVM_DECLARE_FINAL_OBJECT_INFO(BufferLoadNode, PrimExprNode);
+
+ private:
+  /*! \brief Set the dtype based on the buffer/indices
+   *
+   * Usually, the BufferLoad's dtype will be the same dtype as the
+   * buffer.  This may have a different number of lanes than the
+   * buffer's dtype if index values have more than 1 lane.
+   *
+   * This function should only be called during construction and after

Review comment:
       Add TODO, add WithIndices function that uses LegalizeDType, which is called by those friend classes.




-- 
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 merged pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   


-- 
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] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Rebased onto main, resolving merge conflict.


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/merge_dynamic_shared_memory_allocations.cc
##########
@@ -294,22 +308,61 @@ class DynamicSharedMemoryRewriter : public StmtExprMutator {
   }
 
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    if (IsDynamicSharedMemory(op->buffer_var)) {
-      PrimExpr offset = GetBufferOffset(op->buffer_var, op->dtype);
-      PrimExpr index = StmtExprMutator::VisitExpr(op->index);
-      return Load(op->dtype, merged_buf_var_, offset + index, op->predicate, op->span);
-    }
-    return StmtExprMutator::VisitExpr_(op);
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
 
   Stmt VisitStmt_(const StoreNode* op) final {
-    if (IsDynamicSharedMemory(op->buffer_var)) {
-      PrimExpr offset = GetBufferOffset(op->buffer_var, op->value->dtype);
-      PrimExpr index = StmtExprMutator::VisitExpr(op->index);
-      PrimExpr value = StmtExprMutator::VisitExpr(op->value);
-      return Store(merged_buf_var_, value, offset + index, op->predicate, op->span);
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(std::move(node));
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto node = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return VisitBufferAccess(std::move(node));
+  }
+
+  template <typename Node>
+  Node VisitBufferAccess(Node node) {
+    if (IsDynamicSharedMemory(node->buffer->data)) {
+      ICHECK_EQ(node->indices.size(), 1)
+          << "MergeDynamicSharedMemoryAllocations expects flat memory buffers, "
+          << "and is to be run after "
+          << "StorageFlatten (TE schedules) or FlattenBuffer (TIR schedules)";
+      Array<PrimExpr> indices = {node->indices[0] +
+                                 this->GetBufferOffset(node->buffer->data, node->buffer->dtype)};
+
+      auto writer = node.CopyOnWrite();
+      writer->buffer = GetUpdatedBuffer(node->buffer);
+      writer->indices = indices;
     }
-    return StmtExprMutator::VisitStmt_(op);
+
+    return node;
+  }
+
+  Buffer GetUpdatedBuffer(Buffer buffer) {
+    auto key = buffer.get();
+    auto it = buffer_remap_.find(key);
+    if (it != buffer_remap_.end()) {
+      return it->second;
+    }
+
+    if (IsDynamicSharedMemory(buffer->data)) {
+      ICHECK_EQ(buffer->shape.size(), 1)
+          << "MergeDynamicSharedMemoryAllocations expects flat memory buffers, "
+          << "and is to be run after "
+          << "StorageFlatten (TE schedules) or FlattenBuffer (TIR schedules)";
+      auto writer = buffer.CopyOnWrite();
+      writer->data = merged_buf_var_;

Review comment:
       I think the aliasing is necessary overall, because the dynamic shared memory buffers aren't necessarily the same underlying type.  That said, I can imagine a later improvement that would identify buffers of the same type and them merge those, but I didn't want to add complexity for the first time around.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: python/tvm/script/tir/special_stmt.py
##########
@@ -132,6 +132,7 @@ def match_buffer(
             align=-1,
             offset_factor=0,
             buffer_type="default",
+            flatten_buffer=False,

Review comment:
       Thank you for pointing it out, and I ended up removing the argument entirely.  It was an early attempt that was later replaced with `T.preflattened_buffer`, and I should have removed it at that point.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/function.h
##########
@@ -136,15 +157,31 @@ class PrimFunc : public BaseFunc {
  public:
   /*!
    * \brief Constructor
+   *
    * \param params The parameters of the function.
+   *
    * \param body The body of the function.
+   *
    * \param ret_type The return type of the function.
+   *
    * \param buffer_map The buffer map for parameter buffer unpacking.
+   * This contains buffer objects as they appear in the body of the
+   * PrimFunc.  (e.g. a buffer of shape ``[1024]`` originally
+   * generated as a tensor of shape ``[32, 32]``)
+   *
+   * \param preflattened_buffer_map The buffer map for
+   * parameter buffer unpacking.  This contains buffer
+   * objects as they are expected to be passed in by the
+   * callee.  (e.g. a buffer of shape ``[32, 32]`` originally
+   * generated as a tensor of shape ``[32, 32]``)
+   *
    * \param attrs Additional function attributes.
+   *
    * \param span The location of this object in the source code.
    */
   TVM_DLL PrimFunc(Array<tir::Var> params, Stmt body, Type ret_type = VoidType(),
                    Map<tir::Var, Buffer> buffer_map = Map<tir::Var, Buffer>(),
+                   Map<tir::Var, Buffer> preflattened_buffer_map = Map<tir::Var, Buffer>(),

Review comment:
       And updated.




-- 
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] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Merge commit from main to dev done to resolve merge conflicts without impacting in-progress reviews.  Follow-up commits are to resolve breakages in additional unittests that exist in main and were pulled in by the merge.


-- 
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] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Rebasing onto main to resolve merge conflicts.  Apologies in advance for any in-progress reviews that get orphaned by the rebase.


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       I'm fine with it either way, though I think I'd want to have that as a later change.  My one concern is that having the entire pre-flattened Buffer available could allow post-flattening passes to have access to more information than is necessary, and information that may be out of date.  Storing only the minimum amount of information required for later use minimizes the amount of state that must be kept consistent during later passes.




-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/expr.h
##########
@@ -610,6 +610,14 @@ class BufferLoadNode : public PrimExprNode {
   /*! \brief The indices location to be loaded. */
   Array<PrimExpr> indices;
 
+  /*! \brief Set the dtype based on the buffer/indices
+   *
+   * Usually, this will be the same dtype as the buffer.  This may
+   * have a different number of lanes than the buffer's dtype if index
+   * values have more than 1 lane.
+   */
+  void LegalizeDtype();

Review comment:
       Reading again, I now think that LegalizeDtype should be a private function called by CTOR. COW was a convenient way to construct a new field, but this comes with the price of leaking this behavior to the outside. Consider add a function WithIndices(Bufferload, indices) that calls into this legalization function. Can be done as a followup




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/printer/tvmscript_printer.cc
##########
@@ -1008,8 +1013,59 @@ Doc TVMScriptPrinter::VisitStmt_(const BufferRealizeNode* op) {
   return Doc();
 }
 
+namespace {
+struct AllocUsage {
+  Buffer alloc_buffer;
+  Array<Buffer> aliasing_buffers;
+};
+
+template <typename AllocNode>
+AllocUsage find_allocate_usage(AllocNode* op, Map<Var, Array<Buffer>>* cache_ptr) {

Review comment:
       And updated.  For some reason I thought that free functions used snake_case.




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

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 edited a comment on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on pull request #9727:
URL: https://github.com/apache/tvm/pull/9727#issuecomment-1060129410


   Tracking issue of followup shepherding https://github.com/apache/tvm/issues/10505


-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: tests/python/unittest/test_tir_transform_instrument_bound_checkers.py
##########
@@ -94,7 +94,7 @@ def test_out_of_bounds_vectorize_llvm(nn, index_a, index_b):
 @tvm.testing.requires_llvm
 def test_in_bounds_vectorize_llvm():
     n = 512
-    lanes = 2
+    lanes = 1

Review comment:
       is this change needed to pass the test?

##########
File path: include/tvm/tir/buffer.h
##########
@@ -98,10 +113,17 @@ class BufferNode : public Object {
   }
 
   bool SEqualReduce(const BufferNode* other, SEqualReducer equal) const {
-    // Use DefEqual as buffer can define variables
-    // in its semantics, skip name as name is not important.
+    // Use DefEqual as buffer can define variables in its semantics,
+    // skip name as name is not important.
+
+    // The pre-flattened information is only used for type-checking,
+    // and doesn't represent a different computation.
+    //
+    // TODO(Lunderberg): Move the pre-flattened buffer information
+    // into the PrimFunc's buffer_map.

Review comment:
       update the comment as pre-flattened information is no longer here

##########
File path: src/tir/transforms/merge_dynamic_shared_memory_allocations.cc
##########
@@ -294,22 +308,61 @@ class DynamicSharedMemoryRewriter : public StmtExprMutator {
   }
 
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    if (IsDynamicSharedMemory(op->buffer_var)) {
-      PrimExpr offset = GetBufferOffset(op->buffer_var, op->dtype);
-      PrimExpr index = StmtExprMutator::VisitExpr(op->index);
-      return Load(op->dtype, merged_buf_var_, offset + index, op->predicate, op->span);
-    }
-    return StmtExprMutator::VisitExpr_(op);
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
 
   Stmt VisitStmt_(const StoreNode* op) final {
-    if (IsDynamicSharedMemory(op->buffer_var)) {
-      PrimExpr offset = GetBufferOffset(op->buffer_var, op->value->dtype);
-      PrimExpr index = StmtExprMutator::VisitExpr(op->index);
-      PrimExpr value = StmtExprMutator::VisitExpr(op->value);
-      return Store(merged_buf_var_, value, offset + index, op->predicate, op->span);
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(std::move(node));
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto node = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return VisitBufferAccess(std::move(node));
+  }
+
+  template <typename Node>
+  Node VisitBufferAccess(Node node) {
+    if (IsDynamicSharedMemory(node->buffer->data)) {
+      ICHECK_EQ(node->indices.size(), 1)
+          << "MergeDynamicSharedMemoryAllocations expects flat memory buffers, "
+          << "and is to be run after "
+          << "StorageFlatten (TE schedules) or FlattenBuffer (TIR schedules)";
+      Array<PrimExpr> indices = {node->indices[0] +
+                                 this->GetBufferOffset(node->buffer->data, node->buffer->dtype)};
+
+      auto writer = node.CopyOnWrite();
+      writer->buffer = GetUpdatedBuffer(node->buffer);
+      writer->indices = indices;
     }
-    return StmtExprMutator::VisitStmt_(op);
+
+    return node;
+  }
+
+  Buffer GetUpdatedBuffer(Buffer buffer) {
+    auto key = buffer.get();
+    auto it = buffer_remap_.find(key);
+    if (it != buffer_remap_.end()) {
+      return it->second;
+    }
+
+    if (IsDynamicSharedMemory(buffer->data)) {
+      ICHECK_EQ(buffer->shape.size(), 1)
+          << "MergeDynamicSharedMemoryAllocations expects flat memory buffers, "
+          << "and is to be run after "
+          << "StorageFlatten (TE schedules) or FlattenBuffer (TIR schedules)";
+      auto writer = buffer.CopyOnWrite();
+      writer->data = merged_buf_var_;

Review comment:
       This pass consolidates multiple dynamic shared memory buffers into one, shall we also create a buffer `merged_buf_` instead of creating an alias here? 

##########
File path: python/tvm/script/tir/special_stmt.py
##########
@@ -132,6 +132,7 @@ def match_buffer(
             align=-1,
             offset_factor=0,
             buffer_type="default",
+            flatten_buffer=False,

Review comment:
       Update the comment here https://github.com/apache/tvm/blob/e40414fc81d99c235f857ae9c741a4f25d072f79/python/tvm/script/tir/special_stmt.py#L103-L104 and mention the usage of `flatten_buffer`

##########
File path: src/tir/transforms/split_host_device.cc
##########
@@ -155,10 +160,27 @@ class VarUseDefAnalysis : public StmtExprMutator {
   }
 
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    this->HandleUse(op->buffer_var);
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    this->HandleUse(op->buffer->data);
     return StmtExprMutator::VisitExpr_(op);
   }
 
+  void VisitBuffer(Buffer buffer) {
+    this->HandleUse(buffer->data);
+    auto visit_arr = [&](Array<PrimExpr> arr) {
+      for (const auto& element : arr) {
+        this->VisitExpr(element);
+      }
+    };
+
+    visit_arr(buffer->shape);
+    visit_arr(buffer->strides);
+  }
+

Review comment:
       is this used?

##########
File path: include/tvm/tir/function.h
##########
@@ -136,15 +157,31 @@ class PrimFunc : public BaseFunc {
  public:
   /*!
    * \brief Constructor
+   *
    * \param params The parameters of the function.
+   *
    * \param body The body of the function.
+   *
    * \param ret_type The return type of the function.
+   *
    * \param buffer_map The buffer map for parameter buffer unpacking.
+   * This contains buffer objects as they appear in the body of the
+   * PrimFunc.  (e.g. a buffer of shape ``[1024]`` originally
+   * generated as a tensor of shape ``[32, 32]``)
+   *
+   * \param preflattened_buffer_map The buffer map for
+   * parameter buffer unpacking.  This contains buffer
+   * objects as they are expected to be passed in by the
+   * callee.  (e.g. a buffer of shape ``[32, 32]`` originally
+   * generated as a tensor of shape ``[32, 32]``)
+   *
    * \param attrs Additional function attributes.
+   *
    * \param span The location of this object in the source code.
    */
   TVM_DLL PrimFunc(Array<tir::Var> params, Stmt body, Type ret_type = VoidType(),
                    Map<tir::Var, Buffer> buffer_map = Map<tir::Var, Buffer>(),
+                   Map<tir::Var, Buffer> preflattened_buffer_map = Map<tir::Var, Buffer>(),

Review comment:
       use `Optional<Map<tir::Var, Buffer>>` to explicitly mark it as optional




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/split_host_device.cc
##########
@@ -155,10 +160,27 @@ class VarUseDefAnalysis : public StmtExprMutator {
   }
 
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    this->HandleUse(op->buffer_var);
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    this->HandleUse(op->buffer->data);
     return StmtExprMutator::VisitExpr_(op);
   }
 
+  void VisitBuffer(Buffer buffer) {
+    this->HandleUse(buffer->data);
+    auto visit_arr = [&](Array<PrimExpr> arr) {
+      for (const auto& element : arr) {
+        this->VisitExpr(element);
+      }
+    };
+
+    visit_arr(buffer->shape);
+    visit_arr(buffer->strides);
+  }
+

Review comment:
       Thank you, I should have used it from the BufferStore/BufferLoad visitors, which I've now corrected.




-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/function.h
##########
@@ -91,11 +91,23 @@ class PrimFuncNode : public BaseFuncNode {
    */
   Map<tir::Var, Buffer> buffer_map;
 
+  /*! \brief The buffer map prior to flattening.
+   *
+   * This contains the buffers as they exists prior to flattening, and
+   * is used for validating an input tensor passed into the packed
+   * API.  Any buffer that is present in `buffer_map` but not present
+   * in `preflattened_buffer_map` is assumed to be the same before
+   * and after flattening (e.g. a 1-d tensor that is backed by 1-d
+   * flat memory).
+   */
+  Map<tir::Var, Buffer> preflattened_buffer_map;

Review comment:
       What is the reasoning behind preserving this?
   
   The comment sounds like it is for validation. Do we need to maintain a member just for validation or is there more uses than that ?
   

##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,22 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The type of the buffer prior to flattening
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*!
+   * \brief Separators between input axes when generating flattened output axes
+   *
+   * For buffers representing flat 1-d memory (e.g. any buffer in
+   * RAM), this should be an empty array.  For buffers representing
+   * non-flat memory, each entry in axis_separators should be the
+   * first input axis that is part of a new flattened axis.
+   */
+  Array<IntImm> axis_separators;

Review comment:
       Maybe this was discussed elsewhere (in which case please point me at that), what is the reasoning behind maintaining this in the IR. I would naively assume shape to be transformed on-demand basis ?

##########
File path: include/tvm/te/schedule.h
##########
@@ -771,6 +835,36 @@ class Singleton : public IterVarRelation {
   TVM_DEFINE_OBJECT_REF_METHODS(Singleton, IterVarRelation, SingletonNode);
 };
 
+/*!
+ * \brief Transform iterator according to some arbitrary expression.
+ */
+class TransformNode : public IterVarRelationNode {
+ public:
+  Array<IterVar> original_variables;

Review comment:
       Should we have docs for these variables ?




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,22 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The type of the buffer prior to flattening
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*!
+   * \brief Separators between input axes when generating flattened output axes
+   *
+   * For buffers representing flat 1-d memory (e.g. any buffer in
+   * RAM), this should be an empty array.  For buffers representing
+   * non-flat memory, each entry in axis_separators should be the
+   * first input axis that is part of a new flattened axis.
+   */
+  Array<IntImm> axis_separators;

Review comment:
       There are two separate steps to the transformation, one which re-orders the existing indices and a later transformation that produces flat memory buffers.  The former is either on-demand (TIR schedules), or during the building of a PrimFunc (SchedulePostProcToPrimFunc for TE-based schedules).  The latter is applied during the lowering steps, either in StorageFlatten (TE-based schedules) or FlattenBuffer (TIR-based schedules).  We had some discussion ([link to RFC section with some details](https://github.com/Lunderberg/tvm-rfcs/blob/data_layout/rfcs/0039-buffer-physical-layout.md#unresolved-questions), [link to comments on the RFC](https://github.com/apache/tvm-rfcs/pull/0039#issuecomment-961358086)) on having the flattening to 1-d be a special case of layout transformations that are applied on-demand.  However, @vinx13 brought up that some of the TIR lowering steps would need to apply after axis transformation makes the shape of the transformed buffer available, but before bu
 ffer flattening removes that shape information.
   
   Maintaining the `axis_separators` through to the buffer flattening allows the later transformation to produce buffers representing non-flat memory spaces (e.g. texture memory on GPUs).




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/vectorize_loop.cc
##########
@@ -62,30 +62,89 @@ class VecAllocAccess : public StmtExprMutator {
  public:
   VecAllocAccess(const VarNode* buf, Var var, int var_lanes)
       : buf_(buf), var_(var), var_lanes_(var_lanes) {}
-  // Load
+
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    PrimExpr expr = StmtExprMutator::VisitExpr_(op);
-    op = expr.as<LoadNode>();
-    if (op->buffer_var.get() == buf_) {
-      return Load(op->dtype, op->buffer_var, op->index * var_lanes_ + var_, op->predicate);
-    } else {
-      return expr;
-    }
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
-  // Store
+
   Stmt VisitStmt_(const StoreNode* op) final {
-    Stmt stmt = StmtExprMutator::VisitStmt_(op);
-    op = stmt.as<StoreNode>();
-    if (op->buffer_var.get() == buf_) {
-      return Store(op->buffer_var, op->value, op->index * var_lanes_ + var_, op->predicate);
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto load = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return UpdateBufferAccess(load);
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto store = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return UpdateBufferAccess(store);
+  }
+
+ private:
+  template <typename Node>
+  Node UpdateBufferAccess(Node node) {
+    // Only update the buffer that's being replaced.
+    if (node->buffer->data.get() != buf_) {
+      return node;
+    }
+
+    arith::Analyzer analyzer;

Review comment:
       Sounds good, and done.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/target/source/codegen_c.cc
##########
@@ -158,78 +158,52 @@ void CodeGenC::PrintSSAAssign(const std::string& target, const std::string& src,
 }
 
 // Print a reference expression to a buffer.
-std::string CodeGenC::GetBufferRef(DataType t, const VarNode* buffer, PrimExpr index) {
+std::string CodeGenC::GetBufferRef(DataType t, const BufferNode* buffer, PrimExpr index) {
+  const VarNode* buffer_var = buffer->data.get();
   std::ostringstream os;
-  std::string vid = GetVarID(buffer);
+  std::string vid = GetVarID(buffer_var);
   std::string scope;
-  if (alloc_storage_scope_.count(buffer)) {
-    scope = alloc_storage_scope_.at(buffer);
+  if (alloc_storage_scope_.count(buffer_var)) {
+    scope = alloc_storage_scope_.at(buffer_var);
   }
-  bool is_vol = IsVolatile(buffer);
-  if (t.lanes() == 1) {
-    if (!HandleTypeMatch(buffer, t) || is_vol) {
-      os << "((";
-      if (is_vol) {
-        os << "volatile ";
-      }
-      // Scope may not be part of type.
-      if (!scope.empty() && IsScopePartOfType()) {
-        PrintStorageScope(scope, os);
-      }
-      PrintType(t, os);
-      os << "*)" << vid << ')';
-    } else {
-      os << vid;
-    }
-    os << "[(";
-    PrintExpr(index, os);
-    os << ")";
-    if (t.bits() == 4 || (t.bits() == 1 && t.is_int())) {
-      os << " / " << (32 / t.bits());
-    }
-    os << ']';
-  } else {

Review comment:
       It does.  The `(32 / t.bits())` in that branch is to do a computation in terms of the type that has been cast to, rather than the type of the underlying 32-bit integer that backs the array.  This same behavior comes out with the check for the number of lanes.




-- 
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] Lunderberg commented on pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   Fixing lint issues.  The large number of commits listed is due to rebasing the fixes into the commits that introduced them.


-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/te/schedule/schedule_ops.cc
##########
@@ -343,12 +367,16 @@ Stmt ScheduleOps(Schedule sch, Map<IterVar, Range> dom_map_, bool debug_keep_tri
     Stage s = sch->stages[i - 1];
     ICHECK_NE(s->attach_type, kInline) << "call schedule.normalize before scheduleops";
     ICHECK(s->op.defined());
-    // no need to specify place holder op.
-    if (s->op.as<PlaceholderOpNode>()) continue;
     // Remove grouping sugar, get the real attach spec.
     Stage attach_spec = s.GetAttachSpec();
 
-    if (scan_init.count(s->op)) {
+    if (s->op.as<PlaceholderOpNode>()) {
+      // Placeholders don't need any realize/provide statements, but
+      // may be annotated with set_physical_layout to indicate the
+      // physical layout of an input, and must still have the
+      // attribute given.
+      body = WrapPhysicalLayout(s, std::move(body));

Review comment:
       Thank you, and renamed to `WrapLayoutTransformationAttrs`




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: python/tvm/te/schedule.py
##########
@@ -519,9 +523,149 @@ def rolling_buffer(self):
         """
         _ffi_api.StageRollingBuffer(self)
 
+    def transform_layout(self, mapping_function: Callable[..., List[tvm.tir.PrimExpr]]):
+        """Defines the layout transformation for the current stage's tensor.
+
+        The map from initial_indices to final_indices must be an
+        invertible affine transformation.  This method may be called
+        more than once for a given tensor, in which case each
+        transformation is applied sequentially.
+
+        If the stage is a ComputeOp, then the iteration order of the
+        compute stage is rewritten to be a row-major traversal of the
+        tensor, and the new loop iteration variables are returned.
+        For all other stages, the loop iteration order is unmodified,
+        and the return value is None.
+
+        Parameters
+        ----------
+        mapping_function : Callable[..., List[tvm.tir.PrimExpr]]
+
+            A callable that accepts N arguments of type tvm.tir.Var,
+            and outputs a list of PrimExpr.  The input arguments
+            represent the location of a value in the current stage's
+            tensor, using the pre-transformation layout.  The return
+            value of the function gives the location of that value in
+            the current stage's tensor, using the post-transformation
+            layout.
+
+        Returns
+        -------
+        new_iter_vars : Optional[List[tvm.tir.IterVar]]
+
+            If the stage is a ComputeOp, then the return will be the
+            updated loop iteration variables over the data array, in
+            the same order as the output values from the
+            `mapping_function`.
+
+            Otherwise, the return value is None.
+
+        Examples
+        --------
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in NHWC
+            # format, and should be transformed into NCHWc format.
+
+            s[A].transform_layout(
+                lambda n,h,w,c: [n, c//4, h, w, c%4]
+            )
+
+
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in an
+            # arbitrary format, and should be transformed such that
+            # the last index is split, with the slower-changing index
+            # of the split placed at the slowest changing dimension.
+
+            s[A].transform_layout(
+                lambda *indices, i: [i//4, *indices, i%4]

Review comment:
       I'm cheating a little bit here to avoid delaying the production of the IndexMap.  Since the dimensionality of the stage `A` is known, the `IndexMap` corresponding to the lambda function can be created immediately, and doesn't require waiting until later on in the build/lower step.  (e.g. if `len(A.shape)==3`, then this call would generate an `IndexMap` with `initial_indices = [indices0, indices1, i]`, and `final_indices = [i//4, indices0, indices1, i%4]`).  The same lambda function could produce different `IndexMap` objects depending on if it is passed to `s[A].transform_layout` or `s[B].transform_layout`, rather than producing a single 
   
   That said, if I'm wrong about the implications and there are some edge cases that require delaying the `IndexMap` creation, I definitely don't want to require tracking a partially made `IndexMap` through some of the lowering steps.  As it was, this seemed the cleanest way to expose the Python API, and had similarities to how `te.compute`'s interface works.




-- 
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] mbs-octoml commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

Posted by GitBox <gi...@apache.org>.
mbs-octoml commented on a change in pull request #9727:
URL: https://github.com/apache/tvm/pull/9727#discussion_r768910010



##########
File path: include/tvm/te/schedule.h
##########
@@ -466,9 +498,27 @@ class StageNode : public Object {
    *  while origin_op remains fixed.
    */
   Operation origin_op;
-  /*! \brief All the nodes in the iter var */
+  /*! \brief All the nodes in the iter var
+   *
+   * Each element of all_iter_vars represents an iteration variable

Review comment:
       +1 for filling in the missing comments!

##########
File path: include/tvm/tir/stmt.h
##########
@@ -1276,6 +1278,21 @@ constexpr const char* pragma_tensor_core = "pragma_tensor_core";
  *  run prefetch of Tensor on the current loop scope
  */
 constexpr const char* prefetch_scope = "prefetch_scope";
+/*!
+ * \brief Marks the layout transforms to be used for a tensor.
+ *
+ * Only applies to a DataProducer, as it should be made part of the
+ * PrimFunc attributes for TIR.
+ */
+constexpr const char* layout_transforms = "layout_transforms";

Review comment:
       (note to self) would be subsumed by VirtualDevice field I think.

##########
File path: include/tvm/te/schedule.h
##########
@@ -256,6 +257,37 @@ class Stage : public ObjectRef {
    * \return reference to self.
    */
   TVM_DLL Stage& rolling_buffer();  // NOLINT(*)
+  /*!
+   * \brief Defines a layout transformation to be applied to the buffer.
+   *
+   * The map from initial_index to final_index must be an
+   * invertible affine transformation.
+   *
+   * \param initial_indices An array of variables to represent a

Review comment:
       nit: explain these are binding occurrences, ie fresh vars for the purposes of representing the implicit original->new index function.

##########
File path: include/tvm/tir/buffer.h
##########
@@ -127,7 +174,7 @@ class BufferNode : public Object {
    * without adjusting for number of lanes.  (e.g. The number of
    * float16x4 elements in a buffer of type float16x4.)
    */
-  PrimExpr ElemOffset(Array<PrimExpr> index) const;
+  Array<PrimExpr> ElemOffset(Array<PrimExpr> index) const;

Review comment:
       nit: update comment
   
   Would be nice to have aliases for PhysicalIndex, LogicalIndex ?

##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening

Review comment:
       (notes to self)
    - Reminiscent of the need to track the 'Relay type' of PrimFuncs (ie before flattening and conversion to DPS).
    - Perhaps this validation & (presumably) conversion should be pushed into whatever the generalization of device_copy is called

##########
File path: include/tvm/tir/buffer.h
##########
@@ -88,6 +128,8 @@ class BufferNode : public Object {
     v->Visit("data", &data);
     v->Visit("dtype", &dtype);
     v->Visit("shape", &shape);
+    v->Visit("pre_flattened_shape", &pre_flattened_shape);
+    v->Visit("pre_flattened_strides", &pre_flattened_strides);

Review comment:
       don't need to visit the axis_separators or extend eq/hash to them?

##########
File path: include/tvm/tir/index_map.h
##########
@@ -0,0 +1,137 @@
+/*
+ * 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 tvm/tir/index_map.h
+ * \brief Defines a remapping of buffer indices
+ *
+ * For use with tvm::tir::Buffer.
+ */
+#ifndef TVM_TIR_INDEX_MAP_H_
+#define TVM_TIR_INDEX_MAP_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/array.h>
+#include <tvm/runtime/object.h>
+#include <tvm/tir/var.h>
+
+namespace tvm {
+namespace tir {
+
+/*!
+ * \brief Defines the mapping from logical layout of a tensor to

Review comment:
       It's really your core affine shape|->shape representation, and does not need to be bound to log to phys. Eg can be used for any layout xform. I think a few example xforms in the comments to hammer that point home would help, though I see you've done that in the schedule method already.

##########
File path: include/tvm/tir/expr.h
##########
@@ -610,6 +610,14 @@ class BufferLoadNode : public PrimExprNode {
   /*! \brief The indices location to be loaded. */
   Array<PrimExpr> indices;
 
+  /*! \brief Set the dtype based on the buffer/indices
+   *
+   * Usually, this will be the same dtype as the buffer.  This may
+   * have a different number of lanes than the buffer's dtype if index
+   * values have more than 1 lane.
+   */
+  void LegalizeDtype();

Review comment:
       (note to self) why not in ctor? thought  we were trying to avoid mutate-in-place as a rule.

##########
File path: python/tvm/tir/ir_builder.py
##########
@@ -17,10 +17,11 @@
 """Developer API of IR node builder make function."""
 from tvm._ffi.base import string_types
 from tvm.runtime import ObjectGeneric, DataType, convert, const
-from tvm.ir import container as _container, PointerType, PrimType
+from tvm.ir import container as _container

Review comment:
       ------------ up to here ----------------

##########
File path: include/tvm/tir/index_map.h
##########
@@ -0,0 +1,137 @@
+/*
+ * 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 tvm/tir/index_map.h
+ * \brief Defines a remapping of buffer indices
+ *
+ * For use with tvm::tir::Buffer.
+ */
+#ifndef TVM_TIR_INDEX_MAP_H_
+#define TVM_TIR_INDEX_MAP_H_
+
+#include <tvm/ir/expr.h>
+#include <tvm/runtime/container/array.h>
+#include <tvm/runtime/object.h>
+#include <tvm/tir/var.h>
+
+namespace tvm {
+namespace tir {
+
+/*!
+ * \brief Defines the mapping from logical layout of a tensor to
+ * physical layout of a buffer.
+ */
+class IndexMapNode : public Object {

Review comment:
       (note to self) Can this go into Buffer instead of the pre-flattened shapes/strides?
   Should this be the domain for 'layouts' we capture in SEScope (soon to be VirtualDevice) and flow in PlanDevices?
   If so will need structural eq/hash up to alpha conversion, but easy.

##########
File path: python/tvm/te/schedule.py
##########
@@ -519,9 +523,149 @@ def rolling_buffer(self):
         """
         _ffi_api.StageRollingBuffer(self)
 
+    def transform_layout(self, mapping_function: Callable[..., List[tvm.tir.PrimExpr]]):
+        """Defines the layout transformation for the current stage's tensor.
+
+        The map from initial_indices to final_indices must be an
+        invertible affine transformation.  This method may be called
+        more than once for a given tensor, in which case each
+        transformation is applied sequentially.
+
+        If the stage is a ComputeOp, then the iteration order of the
+        compute stage is rewritten to be a row-major traversal of the
+        tensor, and the new loop iteration variables are returned.
+        For all other stages, the loop iteration order is unmodified,
+        and the return value is None.
+
+        Parameters
+        ----------
+        mapping_function : Callable[..., List[tvm.tir.PrimExpr]]
+
+            A callable that accepts N arguments of type tvm.tir.Var,
+            and outputs a list of PrimExpr.  The input arguments
+            represent the location of a value in the current stage's
+            tensor, using the pre-transformation layout.  The return
+            value of the function gives the location of that value in
+            the current stage's tensor, using the post-transformation
+            layout.
+
+        Returns
+        -------
+        new_iter_vars : Optional[List[tvm.tir.IterVar]]
+
+            If the stage is a ComputeOp, then the return will be the
+            updated loop iteration variables over the data array, in
+            the same order as the output values from the
+            `mapping_function`.
+
+            Otherwise, the return value is None.
+
+        Examples
+        --------
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in NHWC
+            # format, and should be transformed into NCHWc format.
+
+            s[A].transform_layout(
+                lambda n,h,w,c: [n, c//4, h, w, c%4]
+            )
+
+
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in an
+            # arbitrary format, and should be transformed such that
+            # the last index is split, with the slower-changing index
+            # of the split placed at the slowest changing dimension.
+
+            s[A].transform_layout(
+                lambda *indices, i: [i//4, *indices, i%4]

Review comment:
       Whoa, ok, I guess this means you 'lazily' produce the IndexMap. Sure you want to commit to 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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       One additional comment.  The pre-flattened information is feeling less and less like a property of the buffer, and is instead feeling like something that should be stored in the `PrimFuncNode::buffer_map`.  It doesn't represent a difference in the computation, it is solely used to validate user input, and it shouldn't be modified at all after flattening occurs.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: tests/python/unittest/test_tir_transform_lower_cross_thread_reduction.py
##########
@@ -25,7 +25,14 @@
 def _check(original, transformed):
     mod = tvm.IRModule.from_expr(original)
     mod = tvm.tir.transform.LowerCrossThreadReduction()(mod)
-    tvm.ir.assert_structural_equal(mod["main"], transformed, True)
+    try:
+        tvm.ir.assert_structural_equal(mod["main"], transformed, True)
+    except ValueError:
+        with open("temp_expected.txt", "w") as f:
+            f.write(transformed.script())
+        with open("temp_observed.txt", "w") as f:
+            f.write(mod["main"].script())

Review comment:
       Thank you, and removed.




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

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

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



[GitHub] [tvm] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/analysis/device_constraint_utils.cc
##########
@@ -223,6 +225,23 @@ class ApplyDeviceConstraintsMutator : public StmtExprMutator {
         any_change = true;
       }
       new_buffer_map.Set(param, new_buffer);
+
+      // Rewrite the pre-flattened buffers to account for constraint.
+      // This only has an impact if the IRModule being analyzed has
+      // already been run through the StorageFlatten or FlattenBuffer
+      // passes.
+      if (auto opt = prim_func->preflattened_buffer_map.Get(param)) {
+        Buffer pf_buffer = opt.value();
+        if (pf_buffer.same_as(buffer)) {

Review comment:
       That's correct.  The preflattened buffer can be the same as the buffer itself if the tensor was 1-d, and is then backed by a flattened 1-d buffer.  The check here is trying to avoid breaking reference equality of `Buffer` objects, should the input primfunc have reference equality.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/te/schedule.h
##########
@@ -256,6 +257,37 @@ class Stage : public ObjectRef {
    * \return reference to self.
    */
   TVM_DLL Stage& rolling_buffer();  // NOLINT(*)
+  /*!
+   * \brief Defines a layout transformation to be applied to the buffer.
+   *
+   * The map from initial_index to final_index must be an
+   * invertible affine transformation.
+   *
+   * \param initial_indices An array of variables to represent a

Review comment:
       Good point, comment has been updated to avoid confusion with passing in the initial `IterVars`.  I also verified that on the python side, there wouldn't be similar confusion.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/expr.h
##########
@@ -610,6 +610,14 @@ class BufferLoadNode : public PrimExprNode {
   /*! \brief The indices location to be loaded. */
   Array<PrimExpr> indices;
 
+  /*! \brief Set the dtype based on the buffer/indices
+   *
+   * Usually, this will be the same dtype as the buffer.  This may
+   * have a different number of lanes than the buffer's dtype if index
+   * values have more than 1 lane.
+   */
+  void LegalizeDtype();

Review comment:
       This is primarily to allow for shared implementations between the `BufferStore` and `BufferLoad` visitors, without needing a dedicated node to represent the buffer access.  See [this comment](https://github.com/apache/tvm-rfcs/pull/42#issuecomment-963314748) for the discussion and an example.
   
   In this case, the `LegalizeDtype` function is called from the constructor, but is also callable after `CopyOnWrite`+mutation.  (e.g. [link](https://github.com/Lunderberg/tvm/blob/1dd14389a/src/tir/transforms/storage_rewrite.cc#L1391))




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/index_map.cc
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 index_map.cc
+ */
+
+#include "tvm/tir/index_map.h"
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/arith/iter_affine_map.h>
+#include <tvm/tir/op.h>
+
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+
+IndexMap::IndexMap(Array<Var> initial_indices, Array<PrimExpr> final_indices) {
+  auto n = make_object<IndexMapNode>();
+  n->initial_indices = std::move(initial_indices);
+  n->final_indices = std::move(final_indices);
+  data_ = std::move(n);
+}
+
+IndexMap IndexMap::Inverse(Array<Range> initial_ranges) const {
+  // Dummy variables to represent the inverse's inputs.
+  Array<Var> output_vars;
+  for (size_t i = 0; i < (*this)->final_indices.size(); i++) {
+    PrimExpr index = (*this)->final_indices[i];
+    // TODO(Lunderberg): Better names for these variables.  A variable
+    // that is passed through unmodified (`index` is an element of
+    // `initial_indices`) should use that input index's name.  A pair
+    // of output indices variables split from a single input index
+    // should be named (X.outer,X.inner).
+    std::stringstream ss;
+    ss << "axis" << i;
+    Var var_index(ss.str(), index.dtype());
+    output_vars.push_back(var_index);
+  }
+
+  // Dummy ranges for the extent of each input.
+  Map<Var, Range> input_iters;
+  ICHECK_EQ((*this)->initial_indices.size(), initial_ranges.size());
+  for (size_t i = 0; i < initial_ranges.size(); i++) {
+    input_iters.Set((*this)->initial_indices[i], initial_ranges[i]);
+  }
+
+  // Unpack the output indices into linear combinations of the initial
+  // indices.
+  arith::Analyzer analyzer;
+  auto iter_map = DetectIterMap((*this)->final_indices, input_iters, 1, true, &analyzer);
+  CHECK(iter_map.size()) << "Index transformation was not bijective.";
+
+  // Determine expressions for the input variables, in terms of the
+  // output variables.
+  Map<Var, PrimExpr> inverse_exprs_map =
+      InverseAffineIterMap(iter_map, Array<PrimExpr>(output_vars.begin(), output_vars.end()));
+
+  // Unpack the map to an array, maintaining the same parameter order.
+  Array<PrimExpr> inverse_exprs;
+  for (const auto& index : (*this)->initial_indices) {
+    inverse_exprs.push_back(inverse_exprs_map.at(index));
+  }
+
+  return IndexMap(output_vars, inverse_exprs);
+}
+
+Array<PrimExpr> IndexMapNode::MapIndices(const Array<PrimExpr>& indices) const {
+  ICHECK_EQ(indices.size(), initial_indices.size());
+
+  arith::Analyzer analyzer;
+
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    analyzer.Bind(initial_indices[i], indices[i]);
+  }
+
+  Array<PrimExpr> output;
+  for (const auto& output_dim : final_indices) {
+    output.push_back(analyzer.Simplify(output_dim));
+  }
+
+  return output;
+}
+
+Array<Range> IndexMapNode::MapRanges(const Array<Range>& ranges) const {
+  ICHECK_EQ(ranges.size(), initial_indices.size());
+
+  Map<Var, Range> input_iters;
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    input_iters.Set(initial_indices[i], ranges[i]);
+  }
+
+  arith::Analyzer analyzer;
+  auto iter_sums = DetectIterMap(final_indices, input_iters, 1, true, &analyzer);
+
+  Array<Range> output;
+  for (const auto& iter_sum : iter_sums) {
+    PrimExpr min = iter_sum->base;
+    PrimExpr extent = 0;
+    for (const auto& term : iter_sum->args) {
+      extent += term->extent * term->scale;
+    }
+    output.push_back(Range::FromMinExtent(min, extent));
+  }
+
+  return output;
+}
+
+Array<PrimExpr> IndexMapNode::MapShape(const Array<PrimExpr>& shape) const {
+  ICHECK_EQ(shape.size(), initial_indices.size());
+
+  Array<Range> ranges;
+  for (auto& dim : shape) {
+    ranges.push_back(Range(0, dim));
+  }
+  Array<Range> mapped = MapRanges(std::move(ranges));
+
+  Array<PrimExpr> output;
+  for (auto& range : mapped) {
+    ICHECK(is_zero(range->min));
+    output.push_back(range->extent);
+  }
+

Review comment:
       shall we also check the size (product of each dimension) is equal before and after the mapping (this is like a simplified check of surjection, or do we allow non-surjective mapping?)

##########
File path: src/tir/transforms/ir_utils.cc
##########
@@ -111,26 +111,56 @@ class IRConvertSSA final : public StmtExprMutator {
       return StmtExprMutator::VisitExpr_(op);
     }
   }
+
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    PrimExpr expr = StmtExprMutator::VisitExpr_(op);
-    op = expr.as<LoadNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Load(op->dtype, scope_[v].back(), op->index, op->predicate);
-    } else {
-      return expr;
-    }
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
+
   Stmt VisitStmt_(const StoreNode* op) final {
-    Stmt stmt = StmtExprMutator::VisitStmt_(op);
-    op = stmt.as<StoreNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Store(scope_[v].back(), op->value, op->index, op->predicate);
-    } else {
-      return stmt;
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(node);
+  }
+
+  Stmt VisitStmt_(const BufferStoreNode* op) final {
+    auto node = Downcast<BufferStore>(StmtExprMutator::VisitStmt_(op));
+    return VisitBufferAccess(node);

Review comment:
       ```suggestion
       return VisitBufferAccess(std::move(node));
   ```

##########
File path: src/tir/transforms/ir_utils.cc
##########
@@ -111,26 +111,56 @@ class IRConvertSSA final : public StmtExprMutator {
       return StmtExprMutator::VisitExpr_(op);
     }
   }
+
   PrimExpr VisitExpr_(const LoadNode* op) final {
-    PrimExpr expr = StmtExprMutator::VisitExpr_(op);
-    op = expr.as<LoadNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Load(op->dtype, scope_[v].back(), op->index, op->predicate);
-    } else {
-      return expr;
-    }
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";
+    return PrimExpr();
   }
+
   Stmt VisitStmt_(const StoreNode* op) final {
-    Stmt stmt = StmtExprMutator::VisitStmt_(op);
-    op = stmt.as<StoreNode>();
-    const VarNode* v = op->buffer_var.get();
-    if (scope_.count(v) && !scope_[v].empty()) {
-      return Store(scope_[v].back(), op->value, op->index, op->predicate);
-    } else {
-      return stmt;
+    LOG(FATAL) << "Unexpected use of deprecated StoreNode.  Please use BufferStoreNode instead.";
+    return Stmt();
+  }
+
+  PrimExpr VisitExpr_(const BufferLoadNode* op) final {
+    auto node = Downcast<BufferLoad>(StmtExprMutator::VisitExpr_(op));
+    return VisitBufferAccess(node);

Review comment:
       ```suggestion
       return VisitBufferAccess(std::move(node));
   ```
   need `std::move` to enable `CopyOnWrite`

##########
File path: src/tir/ir/index_map.cc
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 index_map.cc
+ */
+
+#include "tvm/tir/index_map.h"
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/arith/iter_affine_map.h>
+#include <tvm/tir/op.h>
+
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+
+IndexMap::IndexMap(Array<Var> initial_indices, Array<PrimExpr> final_indices) {
+  auto n = make_object<IndexMapNode>();
+  n->initial_indices = std::move(initial_indices);
+  n->final_indices = std::move(final_indices);
+  data_ = std::move(n);
+}
+
+IndexMap IndexMap::Inverse(Array<Range> initial_ranges) const {
+  // Dummy variables to represent the inverse's inputs.
+  Array<Var> output_vars;
+  for (size_t i = 0; i < (*this)->final_indices.size(); i++) {
+    PrimExpr index = (*this)->final_indices[i];
+    // TODO(Lunderberg): Better names for these variables.  A variable
+    // that is passed through unmodified (`index` is an element of
+    // `initial_indices`) should use that input index's name.  A pair
+    // of output indices variables split from a single input index
+    // should be named (X.outer,X.inner).
+    std::stringstream ss;
+    ss << "axis" << i;
+    Var var_index(ss.str(), index.dtype());
+    output_vars.push_back(var_index);
+  }
+
+  // Dummy ranges for the extent of each input.
+  Map<Var, Range> input_iters;
+  ICHECK_EQ((*this)->initial_indices.size(), initial_ranges.size());
+  for (size_t i = 0; i < initial_ranges.size(); i++) {
+    input_iters.Set((*this)->initial_indices[i], initial_ranges[i]);
+  }
+
+  // Unpack the output indices into linear combinations of the initial
+  // indices.
+  arith::Analyzer analyzer;
+  auto iter_map = DetectIterMap((*this)->final_indices, input_iters, 1, true, &analyzer);
+  CHECK(iter_map.size()) << "Index transformation was not bijective.";
+
+  // Determine expressions for the input variables, in terms of the
+  // output variables.
+  Map<Var, PrimExpr> inverse_exprs_map =
+      InverseAffineIterMap(iter_map, Array<PrimExpr>(output_vars.begin(), output_vars.end()));
+
+  // Unpack the map to an array, maintaining the same parameter order.
+  Array<PrimExpr> inverse_exprs;
+  for (const auto& index : (*this)->initial_indices) {
+    inverse_exprs.push_back(inverse_exprs_map.at(index));
+  }
+
+  return IndexMap(output_vars, inverse_exprs);
+}
+
+Array<PrimExpr> IndexMapNode::MapIndices(const Array<PrimExpr>& indices) const {
+  ICHECK_EQ(indices.size(), initial_indices.size());
+
+  arith::Analyzer analyzer;
+
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    analyzer.Bind(initial_indices[i], indices[i]);
+  }
+
+  Array<PrimExpr> output;
+  for (const auto& output_dim : final_indices) {
+    output.push_back(analyzer.Simplify(output_dim));
+  }
+
+  return output;
+}
+
+Array<Range> IndexMapNode::MapRanges(const Array<Range>& ranges) const {
+  ICHECK_EQ(ranges.size(), initial_indices.size());
+
+  Map<Var, Range> input_iters;
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    input_iters.Set(initial_indices[i], ranges[i]);
+  }
+
+  arith::Analyzer analyzer;
+  auto iter_sums = DetectIterMap(final_indices, input_iters, 1, true, &analyzer);

Review comment:
       Is `DetectIterMap` necessary? This looks like a strong constraint that the mapping should be affine (and is supported by our affine analysis). On the other hand, if we would like to get the range after the mapping, int set analysis will suffice. I'm think about the following alternative:
   ```
     std::unordered_map<const VarNode*, arith::IntSet> dom_map;
     for (size_t i = 0; i < initial_indices.size(); i++) {
        dom_map.emplace(initial_indices[i], IntSet::FromRange(ranges[I]));
     }
     NDIntSet nd_int_set = NDIntSetFromPoint(final_indices);
     nd_int_set = NDIntSetEval(nd_int_set, dom_map);
     Array<Range> output;
     for (const auto& int_set : nd_int_set) {
       output.push_back(Range::FromMinExtent(int_set.min(), int_set.max() - int_set.min() + 1));
     }
   ```

##########
File path: src/te/schedule/schedule_ops.cc
##########
@@ -343,12 +367,16 @@ Stmt ScheduleOps(Schedule sch, Map<IterVar, Range> dom_map_, bool debug_keep_tri
     Stage s = sch->stages[i - 1];
     ICHECK_NE(s->attach_type, kInline) << "call schedule.normalize before scheduleops";
     ICHECK(s->op.defined());
-    // no need to specify place holder op.
-    if (s->op.as<PlaceholderOpNode>()) continue;
     // Remove grouping sugar, get the real attach spec.
     Stage attach_spec = s.GetAttachSpec();
 
-    if (scan_init.count(s->op)) {
+    if (s->op.as<PlaceholderOpNode>()) {
+      // Placeholders don't need any realize/provide statements, but
+      // may be annotated with set_physical_layout to indicate the
+      // physical layout of an input, and must still have the
+      // attribute given.
+      body = WrapPhysicalLayout(s, std::move(body));

Review comment:
       looks like this is a leftover of `physical_layout` renaming

##########
File path: src/target/llvm/codegen_llvm.cc
##########
@@ -931,15 +931,15 @@ llvm::Value* CodeGenLLVM::CreateIntrinsic(const CallNode* op) {
   } else if (op->op.same_as(builtin::tvm_storage_sync())) {
     return CreateStorageSync(op);
   } else if (op->op.same_as(builtin::address_of())) {
-    const LoadNode* l = op->args[0].as<LoadNode>();
-    ICHECK(op->args.size() == 1 && l);
-    TypedPointer buffer_ptr;
-    if (const RampNode* r = l->index.as<RampNode>()) {
-      PrimExpr index = r->base / make_const(DataType::Int(32), r->lanes);
-      buffer_ptr = CreateBufferPtr(l->dtype, MakeValue(l->buffer_var), MakeValue(index));
-    } else {
-      buffer_ptr = CreateBufferPtr(l->dtype, MakeValue(l->buffer_var), MakeValue(l->index));
+    const BufferLoadNode* load = op->args[0].as<BufferLoadNode>();
+    ICHECK(op->args.size() == 1 && load);
+    ICHECK_EQ(load->indices.size(), 0) << "LLVM only supports flat memory allocations.";
+    PrimExpr index = load->indices[0];

Review comment:
       nit: use `const PrimExpr&` to avoid copy

##########
File path: src/target/llvm/codegen_llvm.cc
##########
@@ -1189,14 +1189,24 @@ llvm::Value* CodeGenLLVM::VisitExpr_(const LetNode* op) {
 }
 
 llvm::Value* CodeGenLLVM::VisitExpr_(const LoadNode* op) {
+  LOG(FATAL) << "Unexpected deprecated LoadNode.  Use BufferLoadNode instead.";
+  return NULL;
+}
+
+llvm::Value* CodeGenLLVM::VisitExpr_(const BufferLoadNode* op) {
+  ICHECK_EQ(op->indices.size(), 1) << "CodeGenLLVM expects flattened 1-d buffers.";
+
   DataType t = op->dtype;
-  bool is_volatile = volatile_buf_.count(op->buffer_var.get());
-  llvm::Value* buffer = MakeValue(op->buffer_var);
-  llvm::Value* index = MakeValue(op->index);
+  Var buffer_var = op->buffer->data;
+  PrimExpr buffer_index = op->indices[0];

Review comment:
       const reference




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/lower_custom_datatypes.cc
##########
@@ -103,32 +103,59 @@ class CustomDatatypesLowerer : public StmtExprMutator {
     }
   }
 
-  PrimExpr VisitExpr_(const LoadNode* load) final {
-    bool to_be_lowered = datatype::Registry::Global()->GetTypeRegistered(load->dtype.code());
-    PrimExpr expr = StmtExprMutator::VisitExpr_(load);
-    load = expr.as<LoadNode>();
-    if (to_be_lowered) {
-      auto new_load_type = DataType::UInt(load->dtype.bits());
-      auto buffer_var = load->buffer_var;
-      auto it = var_remap_.find(buffer_var);
-      if (it != var_remap_.end()) {
-        buffer_var = it->second;
-      }
-      return Load(new_load_type, buffer_var, load->index, load->predicate);
-    }
-    return expr;
+  PrimExpr VisitExpr_(const LoadNode* op) final {
+    LOG(FATAL) << "Unexpected use of deprecated LoadNode.  Please use BufferLoadNode instead.";

Review comment:
       comment to this and other passes: is this needed since the base class `StmtMutator` has the default implantation that throws an error

##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       What the plan for this TODO? I feel that keeping a reference to the pre-flattened Buffer can be simpler

##########
File path: src/printer/tir_text_printer.cc
##########
@@ -223,6 +223,9 @@ Doc TIRTextPrinter::BufferNode2Doc(const BufferNode* buf, Doc doc) {
   if (!is_zero(buf->elem_offset)) {
     doc << ", elem_offset=" << Print(buf->elem_offset);
   }
+  if (buf->axis_separators.size()) {
+    doc << ", axis_separators=" << Print(buf->axis_separators);
+  }

Review comment:
       Note on the future work: to support `axis_separator` from tvm script, tvm script printer and the parser also need updates

##########
File path: python/tvm/te/schedule.py
##########
@@ -519,9 +523,149 @@ def rolling_buffer(self):
         """
         _ffi_api.StageRollingBuffer(self)
 
+    def transform_layout(self, mapping_function: Callable[..., List[tvm.tir.PrimExpr]]):
+        """Defines the layout transformation for the current stage's tensor.
+
+        The map from initial_indices to final_indices must be an
+        invertible affine transformation.  This method may be called
+        more than once for a given tensor, in which case each
+        transformation is applied sequentially.
+
+        If the stage is a ComputeOp, then the iteration order of the
+        compute stage is rewritten to be a row-major traversal of the
+        tensor, and the new loop iteration variables are returned.
+        For all other stages, the loop iteration order is unmodified,
+        and the return value is None.
+
+        Parameters
+        ----------
+        mapping_function : Callable[..., List[tvm.tir.PrimExpr]]
+
+            A callable that accepts N arguments of type tvm.tir.Var,
+            and outputs a list of PrimExpr.  The input arguments
+            represent the location of a value in the current stage's
+            tensor, using the pre-transformation layout.  The return
+            value of the function gives the location of that value in
+            the current stage's tensor, using the post-transformation
+            layout.
+
+        Returns
+        -------
+        new_iter_vars : Optional[List[tvm.tir.IterVar]]
+
+            If the stage is a ComputeOp, then the return will be the
+            updated loop iteration variables over the data array, in
+            the same order as the output values from the
+            `mapping_function`.
+
+            Otherwise, the return value is None.
+
+        Examples
+        --------
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in NHWC
+            # format, and should be transformed into NCHWc format.
+
+            s[A].transform_layout(
+                lambda n,h,w,c: [n, c//4, h, w, c%4]
+            )
+
+
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in an
+            # arbitrary format, and should be transformed such that
+            # the last index is split, with the slower-changing index
+            # of the split placed at the slowest changing dimension.
+
+            s[A].transform_layout(
+                lambda *indices, i: [i//4, *indices, i%4]
+            )
+
+        .. code-block:: python
+
+            # ``B`` is a tensor defined by te.compute to be a copy of
+            # ``A`, and should be transformed such that ``B``'s layout
+            # is a transpose of ``A``'s layout.  The loop iteration
+            # that computes ``B`` will correspond to ``B``'s memory
+            # layout.
+
+            A = te.placeholder([n,m])
+            B = te.compute(A.shape, lambda i,j: A[i,j])
+            s = te.create_schedule(B.op)
+
+            s[B].transform_layout(lambda i,j: [j,i])
+
+        """
+
+        args = []
+        var_arg_name = None
+        kwargs = collections.OrderedDict()
+        default_index_dtype = "int32"
+
+        # Make a dummy variable for each explicitly named input index.
+        # We may have some keyword-only arguments, if the function has
+        # *args before the last argument.
+        params = inspect.signature(mapping_function).parameters
+        for name, param in params.items():
+            if param.kind in [
+                inspect.Parameter.POSITIONAL_ONLY,
+                inspect.Parameter.POSITIONAL_OR_KEYWORD,
+            ]:
+                args.append(tvm.tir.Var(name, default_index_dtype))
+
+            elif param.kind == inspect.Parameter.VAR_POSITIONAL:
+                var_arg_name = name
+
+            elif param.kind == inspect.Parameter.KEYWORD_ONLY:
+                kwargs[name] = tvm.tir.Var(name, default_index_dtype)
+
+            elif param.kind in [inspect.Parameter.VAR_KEYWORD]:
+                raise ValueError("transform_layout mapping may not have **kwargs")
+
+        ndim = len(self.op.output(0).shape)
+
+        # Now that all the named arguments have been collected,
+        # everything that remains should go to the *args, if
+        # specified.
+        if var_arg_name is not None:
+            num_var_args = ndim - len(args) - len(kwargs)
+            for i in range(num_var_args):
+                args.append(tvm.tir.Var(f"{var_arg_name}[{i}]", default_index_dtype))
+
+        initial_indices = args + list(kwargs.values())
+        if len(initial_indices) != ndim:
+            raise ValueError(
+                f"transform_layout mapping accepts {len(params)} initial indices, "
+                f"but {self.op.name} is {len(self.op.shape)}-dimensional"
+            )
+
+        mapping = mapping_function(*args, **kwargs)
+
+        final_indices = []
+        axis_separators = []
+        for val in mapping:
+            if isinstance(val, tvm.ir.PrimExpr):
+                final_indices.append(val)
+            elif val is AXIS_SEPARATOR:
+                axis_separators.append(len(final_indices))
+            else:
+                raise TypeError(
+                    "Expected mapping function to return list of "
+                    "either tvm.ir.PrimExpr or tvm.te.AXIS_SEPARATOR.  "
+                    "Instead received {val} of type {type(val)}."
+                )
+
+        new_iter_vars = _ffi_api.StageTransformLayout(self, initial_indices, final_indices)
+        _ffi_api.StageSetAxisSeparators(self, axis_separators)

Review comment:
       Since axis separator is only an attribute to the buffer that affect flattening, would it be better to separate this two API calls to two schedule primitives, even if on the user-facing level? (setting axis separator doesn't have to be involved with layout transform)




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: python/tvm/te/schedule.py
##########
@@ -519,9 +523,149 @@ def rolling_buffer(self):
         """
         _ffi_api.StageRollingBuffer(self)
 
+    def transform_layout(self, mapping_function: Callable[..., List[tvm.tir.PrimExpr]]):
+        """Defines the layout transformation for the current stage's tensor.
+
+        The map from initial_indices to final_indices must be an
+        invertible affine transformation.  This method may be called
+        more than once for a given tensor, in which case each
+        transformation is applied sequentially.
+
+        If the stage is a ComputeOp, then the iteration order of the
+        compute stage is rewritten to be a row-major traversal of the
+        tensor, and the new loop iteration variables are returned.
+        For all other stages, the loop iteration order is unmodified,
+        and the return value is None.
+
+        Parameters
+        ----------
+        mapping_function : Callable[..., List[tvm.tir.PrimExpr]]
+
+            A callable that accepts N arguments of type tvm.tir.Var,
+            and outputs a list of PrimExpr.  The input arguments
+            represent the location of a value in the current stage's
+            tensor, using the pre-transformation layout.  The return
+            value of the function gives the location of that value in
+            the current stage's tensor, using the post-transformation
+            layout.
+
+        Returns
+        -------
+        new_iter_vars : Optional[List[tvm.tir.IterVar]]
+
+            If the stage is a ComputeOp, then the return will be the
+            updated loop iteration variables over the data array, in
+            the same order as the output values from the
+            `mapping_function`.
+
+            Otherwise, the return value is None.
+
+        Examples
+        --------
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in NHWC
+            # format, and should be transformed into NCHWc format.
+
+            s[A].transform_layout(
+                lambda n,h,w,c: [n, c//4, h, w, c%4]
+            )
+
+
+        .. code-block:: python
+
+            # ``A`` is a tensor whose compute definition is in an
+            # arbitrary format, and should be transformed such that
+            # the last index is split, with the slower-changing index
+            # of the split placed at the slowest changing dimension.
+
+            s[A].transform_layout(
+                lambda *indices, i: [i//4, *indices, i%4]
+            )
+
+        .. code-block:: python
+
+            # ``B`` is a tensor defined by te.compute to be a copy of
+            # ``A`, and should be transformed such that ``B``'s layout
+            # is a transpose of ``A``'s layout.  The loop iteration
+            # that computes ``B`` will correspond to ``B``'s memory
+            # layout.
+
+            A = te.placeholder([n,m])
+            B = te.compute(A.shape, lambda i,j: A[i,j])
+            s = te.create_schedule(B.op)
+
+            s[B].transform_layout(lambda i,j: [j,i])
+
+        """
+
+        args = []
+        var_arg_name = None
+        kwargs = collections.OrderedDict()
+        default_index_dtype = "int32"
+
+        # Make a dummy variable for each explicitly named input index.
+        # We may have some keyword-only arguments, if the function has
+        # *args before the last argument.
+        params = inspect.signature(mapping_function).parameters
+        for name, param in params.items():
+            if param.kind in [
+                inspect.Parameter.POSITIONAL_ONLY,
+                inspect.Parameter.POSITIONAL_OR_KEYWORD,
+            ]:
+                args.append(tvm.tir.Var(name, default_index_dtype))
+
+            elif param.kind == inspect.Parameter.VAR_POSITIONAL:
+                var_arg_name = name
+
+            elif param.kind == inspect.Parameter.KEYWORD_ONLY:
+                kwargs[name] = tvm.tir.Var(name, default_index_dtype)
+
+            elif param.kind in [inspect.Parameter.VAR_KEYWORD]:
+                raise ValueError("transform_layout mapping may not have **kwargs")
+
+        ndim = len(self.op.output(0).shape)
+
+        # Now that all the named arguments have been collected,
+        # everything that remains should go to the *args, if
+        # specified.
+        if var_arg_name is not None:
+            num_var_args = ndim - len(args) - len(kwargs)
+            for i in range(num_var_args):
+                args.append(tvm.tir.Var(f"{var_arg_name}[{i}]", default_index_dtype))
+
+        initial_indices = args + list(kwargs.values())
+        if len(initial_indices) != ndim:
+            raise ValueError(
+                f"transform_layout mapping accepts {len(params)} initial indices, "
+                f"but {self.op.name} is {len(self.op.shape)}-dimensional"
+            )
+
+        mapping = mapping_function(*args, **kwargs)
+
+        final_indices = []
+        axis_separators = []
+        for val in mapping:
+            if isinstance(val, tvm.ir.PrimExpr):
+                final_indices.append(val)
+            elif val is AXIS_SEPARATOR:
+                axis_separators.append(len(final_indices))
+            else:
+                raise TypeError(
+                    "Expected mapping function to return list of "
+                    "either tvm.ir.PrimExpr or tvm.te.AXIS_SEPARATOR.  "
+                    "Instead received {val} of type {type(val)}."
+                )
+
+        new_iter_vars = _ffi_api.StageTransformLayout(self, initial_indices, final_indices)
+        _ffi_api.StageSetAxisSeparators(self, axis_separators)

Review comment:
       I think I'd prefer it as a single user-facing call, primarily for readability.  If the two are set in separate API calls, then there would be ambiguity in where the axis separators are placed, which would require the user to read the documentation to determine which convention is used.
   
   ```python
   # Without looking at the documentation, ambiguous whether [1] sets an
   # axis separator before axis 1 or after axis 1.
   s[A].transform_layout(lambda i,j: [i//4, j, i%4])
   s[A].set_axis_separators([1])
   
   # Without looking at the documentation, ambiguous whether [1] sets an
   # axis separator in the pre-transformation axes or the
   # post-transformation axes.
   s[A].set_axis_separators([1])
   s[A].transform_layout(lambda i,j: [j, i//4, i%4])
   
   # Setting both in the same user-facing API call implicitly tells the
   # user that the axis separator occurs between two specific axes, and
   # applies to the transformed axes.
   s[A].transform_layout(lambda i,j: [i//4, j, AXIS_SEPARATOR, i%4])
   ```




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       I like this idea. If this property is only needed for validation, it can be stored in `PrimFuncNode::buffer_map` or maybe `PrimFuncNode::preflattened_buffer_map`




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening
+   *
+   * This contains the shape as it exists prior to flattening, and is
+   * used for validating the shape of the tensor passed into the
+   * packed API.
+   *
+   * TODO(Lunderberg): Should this be a reference to the entire

Review comment:
       It ended up being a relatively straightforward change to add the `PrimFuncNode::preflattened_buffer_map`, which also made it a lot easier to fix some of the breakages caught by `test_lower_build.py`, so that'll be added as another commit on this PR later today.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/expr.cc
##########
@@ -1056,12 +1058,28 @@ TVM_STATIC_IR_FUNCTOR(ReprPrinter, vtable)
     .set_dispatch<AnyNode>([](const ObjectRef& node, ReprPrinter* p) { p->stream << "?"; });
 
 // BufferLoad
+void BufferLoadNode::LegalizeDtype() {
+  int index_lanes = 1;

Review comment:
       Definitely agreed that this is an overgeneralization, and should be restricted further.  One concern with restricting it that much is that it might not handle the case a vector index in an earlier index, which is then moved to the last index by a subsequent layout transformation.
   
   What if we restrict it to have at most one vector index, but allow that index to occur in any location?  That would avoid questions on how multiple vector indices should be flattened (e.g. I could see `[ramp(i,1,4),ramp(j,1,4)]` on a `[4,4]` buffer being reasonable flattened to `[ramp(4*i + j, 1, 4)]` or to `[ramp(ramp(j,1,4), 1,4)]`, or to `[ramp(i, 1, 16)]`, depending on how the two indices interact), but would still allow for the transformed-to-last-index case.




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

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

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



[GitHub] [tvm] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/te/schedule.h
##########
@@ -466,9 +498,27 @@ class StageNode : public Object {
    *  while origin_op remains fixed.
    */
   Operation origin_op;
-  /*! \brief All the nodes in the iter var */
+  /*! \brief All the nodes in the iter var
+   *
+   * Each element of all_iter_vars represents an iteration variable

Review comment:
       Thank you!  I wasn't sure if this should be a separate PR, since it doesn't impact functionality, but it took a bit of code diving to figure out the distinction, and so I wanted to make sure to document it to save time later.  (Including for future-Eric, who is liable to forget these subtleties in a month.)




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/buffer.h
##########
@@ -55,8 +55,48 @@ class BufferNode : public Object {
   Var data;
   /*! \brief data type in the content of the tensor */
   DataType dtype;
-  /*! \brief The shape of the buffer */
+  /*! \brief The shape of the buffer
+   *
+   * This contains the shape as it is accessed by
+   * BufferLoad/BufferStore nodes, and used by the low-level code
+   * generators.
+   */
   Array<PrimExpr> shape;
+  /*! \brief The shape of the buffer prior to flattening

Review comment:
       I agree, and I'd like to have a follow-up discussion at some point on this, as it shows up even at the TE level.  Up until the introduction of the `axis_separators`, there was only a single possible flattening of the buffer to a flat memory space.  With `axis_separators`, there's ambiguity on how a buffer should be flattened, which can result in different storage on the device.  I've sketched out a few possibilities at the TE level, but it should apply similarly when called from Relay, since the extra information to run a generalized device_copy would be needed regardless of the calling interface.
   
   Sketching out a few possibilities, as long as it's in mind.
   
   ```python
   # Current workflow
   dev = tvm.device(TARGET_NAME)
   arr = tvm.nd.array(arr_numpy, device=dev) # host->device copy occurs here
   
   schedule,tensors = make_schedule()
   func = tvm.build(schedule, tensors)
   
   func(arr)
   
   
   # Option 1:
   # - Add axis_separators argument to tvm.nd.array
   # - Con: User must know the axis_separators from prior knowledge.
   # - Con: Requires additional information from calling scope.
   dev = tvm.device(TARGET_NAME)
   arr = tvm.nd.array(arr_numpy, device=dev, axis_separators=[2]) # host->device copy occurs here
   
   schedule,tensors = make_schedule()
   func = tvm.build(schedule, tensors)
   
   func(arr)
   
   # Option 2:
   # - Add for_use_as argument to tvm.nd.array
   # - Con: Requires user to define additional argument.
   dev = tvm.device(TARGET_NAME)
   
   schedule,tensors = make_schedule()
   func = tvm.build(schedule, tensors)
   arr = tvm.nd.array(arr_numpy, device=dev, for_use_as=func.args[0]) # host->device copy occurs here
   
   func(arr)
   
   # Option 3:
   # - Delay the host->device data transfer until the use case is known.
   # - Pro: Doesn't require API change from calling scope.
   # - Con: Does require caching data on the host, in case arr_numpy is
   #   changed between the tvm.nd.array and func calls.
   dev = tvm.device(TARGET_NAME)
   
   schedule,tensors = make_schedule()
   func = tvm.build(schedule, tensors)
   arr = tvm.nd.array(arr_numpy, device=dev) # Cache arr_numpy on host for later use.
   
   func(arr) # host->device copy occurs here
   
   # Option 4:
   
   # - Delay the host->device data transfer until the use case is known,
   #   but only for devices that have non-flat memory regions
   # - Pro: Doesn't require API change from calling scope.
   # - Pro: Doesn't require caching data on the host in all cases.
   # - Con: Adds additional complexity
   dev = tvm.device(TARGET_NAME)
   
   schedule,tensors = make_schedule()
   func = tvm.build(schedule, tensors)
   # For devices that support multi-dimensional memory, arr_numpy is
   # cached here.  For other devices, host->device copy occurs here.
   arr = tvm.nd.array(arr_numpy, device=dev) 
   
   # For devices that support multi-dimensional memory, host->device copy
   # occurs here.
   func(arr)
   ```
   




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/index_map.cc
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 index_map.cc
+ */
+
+#include "tvm/tir/index_map.h"
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/arith/iter_affine_map.h>
+#include <tvm/tir/op.h>
+
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+
+IndexMap::IndexMap(Array<Var> initial_indices, Array<PrimExpr> final_indices) {
+  auto n = make_object<IndexMapNode>();
+  n->initial_indices = std::move(initial_indices);
+  n->final_indices = std::move(final_indices);
+  data_ = std::move(n);
+}
+
+IndexMap IndexMap::Inverse(Array<Range> initial_ranges) const {
+  // Dummy variables to represent the inverse's inputs.
+  Array<Var> output_vars;
+  for (size_t i = 0; i < (*this)->final_indices.size(); i++) {
+    PrimExpr index = (*this)->final_indices[i];
+    // TODO(Lunderberg): Better names for these variables.  A variable
+    // that is passed through unmodified (`index` is an element of
+    // `initial_indices`) should use that input index's name.  A pair
+    // of output indices variables split from a single input index
+    // should be named (X.outer,X.inner).
+    std::stringstream ss;
+    ss << "axis" << i;
+    Var var_index(ss.str(), index.dtype());
+    output_vars.push_back(var_index);
+  }
+
+  // Dummy ranges for the extent of each input.
+  Map<Var, Range> input_iters;
+  ICHECK_EQ((*this)->initial_indices.size(), initial_ranges.size());
+  for (size_t i = 0; i < initial_ranges.size(); i++) {
+    input_iters.Set((*this)->initial_indices[i], initial_ranges[i]);
+  }
+
+  // Unpack the output indices into linear combinations of the initial
+  // indices.
+  arith::Analyzer analyzer;
+  auto iter_map = DetectIterMap((*this)->final_indices, input_iters, 1, true, &analyzer);
+  CHECK(iter_map.size()) << "Index transformation was not bijective.";
+
+  // Determine expressions for the input variables, in terms of the
+  // output variables.
+  Map<Var, PrimExpr> inverse_exprs_map =
+      InverseAffineIterMap(iter_map, Array<PrimExpr>(output_vars.begin(), output_vars.end()));
+
+  // Unpack the map to an array, maintaining the same parameter order.
+  Array<PrimExpr> inverse_exprs;
+  for (const auto& index : (*this)->initial_indices) {
+    inverse_exprs.push_back(inverse_exprs_map.at(index));
+  }
+
+  return IndexMap(output_vars, inverse_exprs);
+}
+
+Array<PrimExpr> IndexMapNode::MapIndices(const Array<PrimExpr>& indices) const {
+  ICHECK_EQ(indices.size(), initial_indices.size());
+
+  arith::Analyzer analyzer;
+
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    analyzer.Bind(initial_indices[i], indices[i]);
+  }
+
+  Array<PrimExpr> output;
+  for (const auto& output_dim : final_indices) {
+    output.push_back(analyzer.Simplify(output_dim));
+  }
+
+  return output;
+}
+
+Array<Range> IndexMapNode::MapRanges(const Array<Range>& ranges) const {
+  ICHECK_EQ(ranges.size(), initial_indices.size());
+
+  Map<Var, Range> input_iters;
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    input_iters.Set(initial_indices[i], ranges[i]);
+  }
+
+  arith::Analyzer analyzer;
+  auto iter_sums = DetectIterMap(final_indices, input_iters, 1, true, &analyzer);

Review comment:
       Good call, and the constraint isn't necessary here.  `DetectIterMap` gets used in `IndexMapNode::Inverse` to check whether we can apply an inverse, but for just the forward transformation it's unnecessary.
   
   Is there an advantage to using the `NDIntSet` object beyond just a normal `IntSet`?  Since we need to have the loop that transforms from an IntSet to a Range in either case, I was thinking it would be more readable to call `arith::EvalSet` directly.
   
   ```c++
   std::unordered_map<const VarNode*, arith::IntSet> dom_map;
     for (size_t i = 0; i < initial_indices.size(); i++) {
       dom_map[initial_indices[i].get()] = arith::IntSet::FromRange(ranges[i]);
     }
   
     Array<Range> output;
     for (const auto& final_index : final_indices) {
       auto int_set = arith::EvalSet(final_index, dom_map);
       output.push_back(Range::FromMinExtent(int_set.min(), int_set.max() - int_set.min() + 1));
     }
   ```




-- 
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] vinx13 commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/index_map.cc
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 index_map.cc
+ */
+
+#include "tvm/tir/index_map.h"
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/arith/iter_affine_map.h>
+#include <tvm/tir/op.h>
+
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+
+IndexMap::IndexMap(Array<Var> initial_indices, Array<PrimExpr> final_indices) {
+  auto n = make_object<IndexMapNode>();
+  n->initial_indices = std::move(initial_indices);
+  n->final_indices = std::move(final_indices);
+  data_ = std::move(n);
+}
+
+IndexMap IndexMap::Inverse(Array<Range> initial_ranges) const {
+  // Dummy variables to represent the inverse's inputs.
+  Array<Var> output_vars;
+  for (size_t i = 0; i < (*this)->final_indices.size(); i++) {
+    PrimExpr index = (*this)->final_indices[i];
+    // TODO(Lunderberg): Better names for these variables.  A variable
+    // that is passed through unmodified (`index` is an element of
+    // `initial_indices`) should use that input index's name.  A pair
+    // of output indices variables split from a single input index
+    // should be named (X.outer,X.inner).
+    std::stringstream ss;
+    ss << "axis" << i;
+    Var var_index(ss.str(), index.dtype());
+    output_vars.push_back(var_index);
+  }
+
+  // Dummy ranges for the extent of each input.
+  Map<Var, Range> input_iters;
+  ICHECK_EQ((*this)->initial_indices.size(), initial_ranges.size());
+  for (size_t i = 0; i < initial_ranges.size(); i++) {
+    input_iters.Set((*this)->initial_indices[i], initial_ranges[i]);
+  }
+
+  // Unpack the output indices into linear combinations of the initial
+  // indices.
+  arith::Analyzer analyzer;
+  auto iter_map = DetectIterMap((*this)->final_indices, input_iters, 1, true, &analyzer);
+  CHECK(iter_map.size()) << "Index transformation was not bijective.";
+
+  // Determine expressions for the input variables, in terms of the
+  // output variables.
+  Map<Var, PrimExpr> inverse_exprs_map =
+      InverseAffineIterMap(iter_map, Array<PrimExpr>(output_vars.begin(), output_vars.end()));
+
+  // Unpack the map to an array, maintaining the same parameter order.
+  Array<PrimExpr> inverse_exprs;
+  for (const auto& index : (*this)->initial_indices) {
+    inverse_exprs.push_back(inverse_exprs_map.at(index));
+  }
+
+  return IndexMap(output_vars, inverse_exprs);
+}
+
+Array<PrimExpr> IndexMapNode::MapIndices(const Array<PrimExpr>& indices) const {
+  ICHECK_EQ(indices.size(), initial_indices.size());
+
+  arith::Analyzer analyzer;
+
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    analyzer.Bind(initial_indices[i], indices[i]);
+  }
+
+  Array<PrimExpr> output;
+  for (const auto& output_dim : final_indices) {
+    output.push_back(analyzer.Simplify(output_dim));
+  }
+
+  return output;
+}
+
+Array<Range> IndexMapNode::MapRanges(const Array<Range>& ranges) const {
+  ICHECK_EQ(ranges.size(), initial_indices.size());
+
+  Map<Var, Range> input_iters;
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    input_iters.Set(initial_indices[i], ranges[i]);
+  }
+
+  arith::Analyzer analyzer;
+  auto iter_sums = DetectIterMap(final_indices, input_iters, 1, true, &analyzer);

Review comment:
       there is nothing special about `NDIntSet`, I agree that using `IntSet` here is more readable




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/ir/index_map.cc
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 index_map.cc
+ */
+
+#include "tvm/tir/index_map.h"
+
+#include <tvm/arith/analyzer.h>
+#include <tvm/arith/iter_affine_map.h>
+#include <tvm/tir/op.h>
+
+#include <sstream>
+
+namespace tvm {
+namespace tir {
+
+IndexMap::IndexMap(Array<Var> initial_indices, Array<PrimExpr> final_indices) {
+  auto n = make_object<IndexMapNode>();
+  n->initial_indices = std::move(initial_indices);
+  n->final_indices = std::move(final_indices);
+  data_ = std::move(n);
+}
+
+IndexMap IndexMap::Inverse(Array<Range> initial_ranges) const {
+  // Dummy variables to represent the inverse's inputs.
+  Array<Var> output_vars;
+  for (size_t i = 0; i < (*this)->final_indices.size(); i++) {
+    PrimExpr index = (*this)->final_indices[i];
+    // TODO(Lunderberg): Better names for these variables.  A variable
+    // that is passed through unmodified (`index` is an element of
+    // `initial_indices`) should use that input index's name.  A pair
+    // of output indices variables split from a single input index
+    // should be named (X.outer,X.inner).
+    std::stringstream ss;
+    ss << "axis" << i;
+    Var var_index(ss.str(), index.dtype());
+    output_vars.push_back(var_index);
+  }
+
+  // Dummy ranges for the extent of each input.
+  Map<Var, Range> input_iters;
+  ICHECK_EQ((*this)->initial_indices.size(), initial_ranges.size());
+  for (size_t i = 0; i < initial_ranges.size(); i++) {
+    input_iters.Set((*this)->initial_indices[i], initial_ranges[i]);
+  }
+
+  // Unpack the output indices into linear combinations of the initial
+  // indices.
+  arith::Analyzer analyzer;
+  auto iter_map = DetectIterMap((*this)->final_indices, input_iters, 1, true, &analyzer);
+  CHECK(iter_map.size()) << "Index transformation was not bijective.";
+
+  // Determine expressions for the input variables, in terms of the
+  // output variables.
+  Map<Var, PrimExpr> inverse_exprs_map =
+      InverseAffineIterMap(iter_map, Array<PrimExpr>(output_vars.begin(), output_vars.end()));
+
+  // Unpack the map to an array, maintaining the same parameter order.
+  Array<PrimExpr> inverse_exprs;
+  for (const auto& index : (*this)->initial_indices) {
+    inverse_exprs.push_back(inverse_exprs_map.at(index));
+  }
+
+  return IndexMap(output_vars, inverse_exprs);
+}
+
+Array<PrimExpr> IndexMapNode::MapIndices(const Array<PrimExpr>& indices) const {
+  ICHECK_EQ(indices.size(), initial_indices.size());
+
+  arith::Analyzer analyzer;
+
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    analyzer.Bind(initial_indices[i], indices[i]);
+  }
+
+  Array<PrimExpr> output;
+  for (const auto& output_dim : final_indices) {
+    output.push_back(analyzer.Simplify(output_dim));
+  }
+
+  return output;
+}
+
+Array<Range> IndexMapNode::MapRanges(const Array<Range>& ranges) const {
+  ICHECK_EQ(ranges.size(), initial_indices.size());
+
+  Map<Var, Range> input_iters;
+  for (size_t i = 0; i < initial_indices.size(); i++) {
+    input_iters.Set(initial_indices[i], ranges[i]);
+  }
+
+  arith::Analyzer analyzer;
+  auto iter_sums = DetectIterMap(final_indices, input_iters, 1, true, &analyzer);

Review comment:
       Sounds good, and updated.




-- 
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] jroesch commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/inject_virtual_thread.cc
##########
@@ -354,46 +391,44 @@ class VTInjector : public StmtExprMutator {
   }
   // Allocate
   Stmt VisitStmt_(const AllocateNode* op) final {
+    Allocate node = GetRef<Allocate>(op);
+
     PrimExpr condition = this->VisitExpr(op->condition);
+
+    Array<PrimExpr> extents = op->extents;
+    extents.MutateByApply([this](const PrimExpr& extent) { return this->VisitExpr(extent); });
+
     if (visit_touched_var_ && !vt_loop_injected_) {
       return InjectVTLoop(GetRef<Stmt>(op), true);
     }
 
-    bool changed = false;
-    Array<PrimExpr> extents;
-    for (size_t i = 0; i < op->extents.size(); i++) {
-      PrimExpr new_ext = this->VisitExpr(op->extents[i]);
-      if (visit_touched_var_ && !vt_loop_injected_) {
-        return InjectVTLoop(GetRef<Stmt>(op), true);
-      }
-      if (!new_ext.same_as(op->extents[i])) changed = true;
-      extents.push_back(new_ext);
-    }
     visit_touched_var_ = false;
 
-    Stmt body;
-    // always rewrite if not allow sharing.
+    // Rewrite the buffer if its shape or any value stored in it
+    // depends on the virtual thread var.  If `allow_share_` is false,
+    // then the buffer is always rewritten, even if separate virtual
+    // threads only read from the buffer.
     if (touched_var_.count(op->buffer_var.get()) || !allow_share_) {
       // place v on highest dimension.
-      PrimExpr stride = foldl([](PrimExpr a, PrimExpr b, Span span) { return mul(a, b, span); },
-                              make_const(DataType::Int(32), 1), op->extents) *
-                        op->dtype.lanes();
-      Array<PrimExpr> other;
-      other.push_back(make_const(op->extents[0].dtype(), num_threads_));
-      for (PrimExpr e : extents) {
-        other.push_back(e);
-      }
-      extents = other;
-      changed = true;
-      // mark this buffer get touched.
+
+      // TODO(Lunderberg): Move pass to apply before

Review comment:
       For these TODOs can we make sure we have some kind of tracking issue/roadmap issue cc @denise-k on this one?




-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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


   followup https://github.com/apache/tvm/issues/10505


-- 
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 #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: include/tvm/tir/builtin.h
##########
@@ -105,10 +105,10 @@ TVM_DLL const Op& large_uint_imm();
 TVM_DLL const Op& q_multiply_shift();
 
 /*!
- * \brief See pesudo code
+ * \brief See pseudo code
  *
- *  Handle address_of(Load *op) {
- *     return &op->buffer_var[index];

Review comment:
       Thanks @Lunderberg To elaborate a bit further, `address_of` is a very low-level primitive and appears at a later stage of the buffer. 
   
   I believe what we are trying to say instead is that "there is a need to build a mechanism to take possible buffer subregion's head address so we can plug them into intrinsics"
   
   While we would find need of taking address of multiple dimensional buffers, the specific design of the instrinsic may not be very desirable at early stage of scheduling, especially when it comes to the case when buffer layout and overall allocation regions changes.
   
   Consider the following example:
   
   ```python
   @T.prim_func
   def myfunc(A: T.Buffer([4,4], "float32"), C: T.Buffer([4,4], "float32")):
        B = T.alloc_buffer([4,4], "float32")
        for i, j in T.grid(2, 2):
              load_2x2_matrix(address_of(B[i*2, j*2], A, i*2, j*2)
              exp_2x2_matrix(address_of(B[i*2, j*2])
              store_2x2_matrix(address_of(B[i*2, j*2], C, i*2, j*2)
   ```
   In this case, the original intention was to load a 2x2 region onto B, perform some arithmetic operations, then store it back to C.  The main problem of this program is how does the address changes as we start to transform the program, say if we swap i and j axis of B, or in this case, what if we compact the layout of B to make it 2x2(as only 2x2 region is touched in each of the loop iterator).
   
   Because `address_of` is a very low level construct, it is hard to clarify what does it mean to update the IR so that the address_of still makes sense. Additionally, there are also some unspoken cosntraints, for example, in many intrinsics there is a restriction that the submatrix have to stay continguous in memory. Which means B have to follow layout `lambda i, j: i//2, j // 2, i %2, j%2` to be lowered.
   
   The high level message is that `address_of` alone is not enough to capture the **intention** of slicing a submatrix with possible addressing constraints. TIR introduced a specific construct (buffer subregion match) to address this problem. Which matches a subregion to a new buffer, and the buffer specified all the possible constraints that backend might want to impose.
   
   Process-wise, restricting the semantics to match the existing behavior would meet our need, but also not open up doors for possible future problems. It could be possible that after some deliberation we still think that `address_of` for multi-dimensional access is needed, then we can update and generalize at that time point
    




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/arg_binder.cc
##########
@@ -184,38 +194,41 @@ void ArgBinder::BindDLTensor(const Buffer& buffer, const PrimExpr& device_type,
                                      IntImm(DataType::Int(32), buffer->data_alignment), nop));
   }
 
+  // shape field
+  Buffer buf_shape = decl_buffer({IntImm(DataType::Int(32), pre_flattened_shape.size())},
+                                 tvm_shape_type, arg_name + ".shape");
   Var v_shape(arg_name + ".shape", DataType::Handle());
   def_handle_dtype_.Set(v_shape, make_const(tvm_shape_type, 0));
   init_nest_.emplace_back(
-      LetStmt(v_shape, TVMArrayGet(DataType::Handle(), handle, builtin::kArrShape), nop));
-  for (size_t k = 0; k < buffer->shape.size(); ++k) {
+      LetStmt(buf_shape->data, TVMArrayGet(DataType::Handle(), handle, builtin::kArrShape), nop));
+  for (size_t k = 0; k < pre_flattened_shape.size(); ++k) {
     if (dtype == DataType::Int(4) || dtype == DataType::UInt(4) || dtype == DataType::Int(1)) {
       break;
     }
     std::ostringstream field_name;
     field_name << v_shape->name_hint << '[' << k << ']';
-    Bind_(buffer->shape[k],
-          cast(buffer->shape[k].dtype(),
-               Load(tvm_shape_type, v_shape, IntImm(DataType::Int(32), k), const_true(1))),
-          field_name.str(), true);
+    Bind_(
+        pre_flattened_shape[k],
+        cast(pre_flattened_shape[k].dtype(), BufferLoad(buf_shape, {IntImm(DataType::Int(32), k)})),
+        field_name.str(), true);
   }
   // strides field
-  Var v_strides(arg_name + ".strides", DataType::Handle());
-  def_handle_dtype_.Set(v_strides, tir::TypeAnnotation(tvm_shape_type));
-  init_nest_.emplace_back(
-      LetStmt(v_strides, TVMArrayGet(DataType::Handle(), handle, builtin::kArrStrides), nop));
-  PrimExpr v_strides_is_null = Call(DataType::Bool(1), builtin::isnullptr(), {v_strides});
-  if (buffer->strides.size() == 0) {
+  Buffer buf_strides = decl_buffer({IntImm(DataType::Int(32), pre_flattened_strides.size())},
+                                   tvm_shape_type, arg_name + ".strides");

Review comment:
       Certainly, and helper function added to generate the names of the handles and elements of the shapes and strides arrays.




-- 
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] Lunderberg commented on a change in pull request #9727: [TE][TIR] Implement layout transformations, non-flat memory buffers

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



##########
File path: src/tir/transforms/inject_virtual_thread.cc
##########
@@ -354,46 +391,44 @@ class VTInjector : public StmtExprMutator {
   }
   // Allocate
   Stmt VisitStmt_(const AllocateNode* op) final {
+    Allocate node = GetRef<Allocate>(op);
+
     PrimExpr condition = this->VisitExpr(op->condition);
+
+    Array<PrimExpr> extents = op->extents;
+    extents.MutateByApply([this](const PrimExpr& extent) { return this->VisitExpr(extent); });
+
     if (visit_touched_var_ && !vt_loop_injected_) {
       return InjectVTLoop(GetRef<Stmt>(op), true);
     }
 
-    bool changed = false;
-    Array<PrimExpr> extents;
-    for (size_t i = 0; i < op->extents.size(); i++) {
-      PrimExpr new_ext = this->VisitExpr(op->extents[i]);
-      if (visit_touched_var_ && !vt_loop_injected_) {
-        return InjectVTLoop(GetRef<Stmt>(op), true);
-      }
-      if (!new_ext.same_as(op->extents[i])) changed = true;
-      extents.push_back(new_ext);
-    }
     visit_touched_var_ = false;
 
-    Stmt body;
-    // always rewrite if not allow sharing.
+    // Rewrite the buffer if its shape or any value stored in it
+    // depends on the virtual thread var.  If `allow_share_` is false,
+    // then the buffer is always rewritten, even if separate virtual
+    // threads only read from the buffer.
     if (touched_var_.count(op->buffer_var.get()) || !allow_share_) {
       // place v on highest dimension.
-      PrimExpr stride = foldl([](PrimExpr a, PrimExpr b, Span span) { return mul(a, b, span); },
-                              make_const(DataType::Int(32), 1), op->extents) *
-                        op->dtype.lanes();
-      Array<PrimExpr> other;
-      other.push_back(make_const(op->extents[0].dtype(), num_threads_));
-      for (PrimExpr e : extents) {
-        other.push_back(e);
-      }
-      extents = other;
-      changed = true;
-      // mark this buffer get touched.
+
+      // TODO(Lunderberg): Move pass to apply before

Review comment:
       Can do.  I've drafted the collected together these TODOs into a list of passes that could be simplified by operating on pre-flattened buffers, along with a high-level description of the changes that would be needed for each.
   
   * InjectVirtualThread
   
       Currently operates on a flattened buffer.  If each virtual thread
       needs its own buffer, the buffer size is increased by a factor of
       `num_threads_`, and all accesses to that buffer are rewritten from
       `i` to `i + vthread_id * original_size`.
   
       Would be simpler to perform before buffer flattening.  If each
       virtual thread needs its own buffer, the buffer's shape is changed
       from `shape` to `[num_threads_, *shape]`, and all accesses to that
       buffer are rewritten from `indices` to `[vthread_id, *indices]`.
   
   * VectorizeLoop
   
       Currently operates on a flattened buffer.  If each iteration of a
       vectorized loop needs its own buffer, the buffer size is increased
       by a factor of `var_lanes_`, and all accesses to that buffer are
       rewritten from `i` to `i*var_lanes_ + var_`.
   
       Would be simpler to perform before buffer flattening.  If each
       iteration of a vectorized loop needs its own buffer, the buffer's
       shape is changed from `shape` to `[*shape, var_lanes_]`, and all
       accesses to that buffer are rewritten from `indices` to
       `[*indices, var_]`.
   
   * InjectDoubleBuffer
   
       Currently operates on a flattened buffer.  The size of the buffer
       is doubled, all reads to that buffer are rewritten from `i` to
       `i + original_size*switch_read_var`, and all writes are rewritten
       from `i` to `i + original_size*switch_write_var`.
   
       Would be simpler to perform before buffer flattening.  The
       buffer's shape is changed from `shape` to `[2, *shape]`, and all
       accesses to the buffer are rewritten from `indices` to
       `[loop_iter%2, *indices]`.
   
   * BoundChecker
   
       Currently operates on a flattened buffer, using attributes
       containing the pre-flattened buffer size.  Would be simpler to
       perform before buffer flattening, using the size of the buffer.
   
   * InjectCopyIntrin
   
       Currently operates on a flattened buffer.  To detect if this is a
       memcpy that can be replaced by an intrinsic, the index of the
       store/load is broken up into linear combinations of the
       surrounding loop iteration variables, then tested to see if those
       strides are consistent with copying each data value sequentially.
       
       Would be simpler to perform before buffer flattening.  If the
       expression a memcpy that can be replaced by an intrinsic, the
       indices of the store/load should be exactly the loop iteration
       variables, and the loop iteration extent should be the size of the
       buffer.




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