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/14 19:02:12 UTC

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

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