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 2022/05/03 02:28:58 UTC

[GitHub] [tvm] jwfromm opened a new pull request, #11199: [Relay] Flexible shape dispatch transformation

jwfromm opened a new pull request, #11199:
URL: https://github.com/apache/tvm/pull/11199

   This PR adds a new pass to `relay.transform` that creates a dispatcher around an input module to handle multiple input shapes. For example consider a case where I'd like to optimize my model to handle both `batch_size=1` and `batch_size=4`. I can now do so elegantly as follows:
   ```
   shape_dict = {'data': [1, 3, 224, 224]}
   model_bs1 = tvmc.load('my_model.onnx', shape_dict=shape_dict)
   tvmc.tune(model, log_file='batch_1.logs')
   
   shape_dict = {'data': [4, 3, 224, 224]}
   model_bs4 = tvmc.load('my_model.onnx', shape_dict=shape_dict)
   tvmc.tune(model, log_file='batch_4.logs')
   
   # Create dispatcher for multiple batch sizes.
   flex_mod = relay.transform.FlexibleShapeDispatch(buckets=[1, 4])(model_bs1.mod)
   
   with ApplyHistoryBest(['batch_1.logs', 'batch_4.logs']):
       exe = relay.vm.compile(flex_mod, "llvm")
   
   # Now we can run inputs with either batch 1 or batch 4 and get the tuned performance!
   batch_1 = np.random.rand(1, 3, 224, 224).astype("float32")
   vm.benchmark(tvm.cpu(), batch_2, func_name="main")
   
   batch_4 = np.random.rand(4, 3, 224, 224).astype("float32")
   vm.benchmark(tvm.cpu(), batch_4, func_name="main")
   ```
   
   As seen above `FlexibleShapeDispatch` is a simple halfway point between fully static and fully dynamic graphs that allows us to leverage TVM tuning. If an input shape is not provided in `buckets`, it will either run fully dynamically using `relay.Any`, or if the `auto_pad` argument is set for `FlexibleShapeDispatch`, padding will be applied to match the closest bucket.
   
   To make applying tuning logs more convenient, I also added the ability to load and merge multiple files to both autotvm and autoscheduler.
   
   Thanks @jroesch for providing the backbone of this implementation.


-- 
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 diff in pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jroesch commented on code in PR #11199:
URL: https://github.com/apache/tvm/pull/11199#discussion_r863735778


##########
python/tvm/relay/transform/flexible_shape.py:
##########
@@ -0,0 +1,273 @@
+# 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.
+# pylint: disable=invalid-name, dangerous-default-value
+"""Relay functions for wrapping a module with flexible shape dispatch."""
+from tvm import relay
+
+
+def override_shape(tensor_type, dim, value):
+    """Change a value in a tensor shape."""
+    new_dims = list(tensor_type.shape)
+    new_dims[dim] = value
+    return relay.TensorType(new_dims, tensor_type.dtype)
+
+
+def specialize_body(mod, function, dim, value, input_indices=[0], affects_output=True):

Review Comment:
   You probably don't want to use the default for input_indices here, default values in Python are allocated precisely once, if you accidentally mutate this anywhere the mutations will be visible by *all invocations* of this function. 



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

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

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


[GitHub] [tvm] jwfromm commented on pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jwfromm commented on PR #11199:
URL: https://github.com/apache/tvm/pull/11199#issuecomment-1116336530

   @jroesch I think the documentation should now be substantially improved based on your input. Let me know what you think.


-- 
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] jwfromm commented on pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jwfromm commented on PR #11199:
URL: https://github.com/apache/tvm/pull/11199#issuecomment-1117486257

   All feedback has been addressed and tests are green. @jroesch do you want to give this one more pass?


-- 
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 diff in pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jroesch commented on code in PR #11199:
URL: https://github.com/apache/tvm/pull/11199#discussion_r863737379


##########
python/tvm/relay/transform/flexible_shape.py:
##########
@@ -0,0 +1,273 @@
+# 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.
+# pylint: disable=invalid-name, dangerous-default-value
+"""Relay functions for wrapping a module with flexible shape dispatch."""
+from tvm import relay
+
+
+def override_shape(tensor_type, dim, value):
+    """Change a value in a tensor shape."""
+    new_dims = list(tensor_type.shape)
+    new_dims[dim] = value
+    return relay.TensorType(new_dims, tensor_type.dtype)
+
+
+def specialize_body(mod, function, dim, value, input_indices=[0], affects_output=True):
+    """Create a subgraph to handle specific input shapes"""
+    # Iterate through specified inputs and construct specialized shapes for each.
+    new_params = list(function.params)
+    data_binding = {}
+    dyn_data_array = []
+    for inp in input_indices:
+        data = function.params[inp]
+        flex_ty = override_shape(data.type_annotation, dim, value)
+        dyn_data = relay.Var(data.name_hint, type_annotation=flex_ty)
+        new_params[inp] = dyn_data
+        data_binding[data] = dyn_data
+        dyn_data_array.append(dyn_data)
+
+    # Create a new function body for the modified shapes.
+    new_body = relay.expr.bind(function.body, data_binding)
+    # Only change the output shape if the input shape affects it.
+    if affects_output:
+        new_ret_ty = override_shape(function.ret_type, dim, value)
+    else:
+        new_ret_ty = function.ret_type
+    gvar = relay.GlobalVar("main_" + str(value))
+    # Add the new function to the main IRModule.
+    mod[gvar] = relay.Function(
+        new_params, new_body, new_ret_ty, function.type_params, function.attrs
+    )
+    return gvar, [d.type_annotation for d in dyn_data_array]
+
+
+def flexible_dispatch(
+    mod, dim=0, buckets=[1], auto_pad=False, pad_value=0, input_indices=[0], affects_output=True
+):
+    """
+    Enable inference of multiple shaped inputs in one module.
+
+    This transformation adds a handler around a module that
+    checks input shapes and dispatches to a subgraph specialized
+    to handle the specific shapes of that input. If no exactly matching
+    subgraph is available, the input will be run using full dynamism.
+    For best performance, specify all the sizes the module will
+    be likely to see using the buckets argument.
+
+    Parameters
+    ----------
+    dim: int
+        The dimension of the input that should be made flexible. This will
+        most often be used for the batch dimension.
+    buckets: list[int]
+        The sizes of the input dimension that should be explicitly handled.
+        Each value in buckets will have a corresponding subgraph constructed to
+        handle it.
+    auto_pad: Optional[bool]
+        If True, then padding will be inserted to values that don't match one of
+        the provided buckets.
+    pad_value: Optional[float]
+        When auto_pad is true, padding will be done with this value.
+    input_indices: Optional[List[int]]
+        Which inputs should be dispatched dynamically, provided by index. All inputs
+        must share the same dynamic axis.
+    affects_output: Optional[bool]
+        Whether the change in input shape has a corresponding effect on the output shape.
+        Batching for example effects both the input and output whereas changing sequence
+        length in an NLP model typically does not.
+
+    Returns
+    -------
+    mod : IRModule
+        The new module wrapped with a flexible shape dispatch handler.
+    """
+    main_fn = mod["main"]
+
+    # Extract all input data and create a new dynamic variable for each.
+    data = []
+    dyn_data = []
+    for i in input_indices:
+        data.append(main_fn.params[i])
+        dyn_shape = override_shape(data[i].type_annotation, dim, relay.Any())
+        dyn_data.append(relay.Var(data[i].name_hint, type_annotation=dyn_shape))
+
+    # Extract the dynamic shape value from one of the inputs.
+    rt_sh = relay.op.shape_of(dyn_data[0])
+    flex_value = relay.op.take(rt_sh, relay.const(dim))
+
+    if_exprs = []
+
+    for i, bucket in enumerate(buckets):
+        input_data = dyn_data
+        check_dim = flex_value
+
+        # Apply automatic padding if specified.
+        if auto_pad:
+            input_data = []
+            # Construct padding expression for inputs.
+            for j, inp in enumerate(dyn_data):
+                pad_width = relay.const(bucket) - flex_value
+                rank = len(data[j].type_annotation.shape)
+                pads = relay.zeros([rank, 2], "int32")
+                pads = relay.scatter_nd(pads, relay.const([dim, 1]), pad_width)
+                padded_value = relay.nn.pad(inp, pads, pad_value)
+
+                # Determine if this is the proper bucket to pad to. Do this by checking if the
+                # input shape is between this bucket and the previous.
+                if i == 0:
+                    padded_value = relay.If(
+                        relay.op.less_equal(flex_value, relay.const(bucket)), padded_value, inp
+                    )
+                else:
+                    padded_value = relay.If(
+                        relay.op.logical_and(
+                            relay.op.less_equal(flex_value, relay.const(bucket)),
+                            relay.op.greater(flex_value, relay.const(buckets[i - 1])),
+                        ),
+                        padded_value,
+                        inp,
+                    )
+                # Update input value and test dimension to reflect possible padding.
+                input_data.append(padded_value)
+            # Grab the new possibly padded shape for checking bucket size.
+            check_dim = relay.op.take(relay.op.shape_of(input_data[0]), relay.const(dim))
+
+        # Create a specialized subgraph for the current bucket.
+        spec_call, spec_ty = specialize_body(
+            mod, main_fn, dim, bucket, input_indices=input_indices, affects_output=affects_output
+        )
+        # Apply hard casting to shape to create statically typed graphs.
+        spec_data = []
+        for j, inp in enumerate(input_data):
+            spec_data.append(relay.op.reshape(inp, spec_ty[j].shape))
+
+        # Create a dispatch statement for the current specialized graph.
+        call_args = list(main_fn.params)
+        for j, inp in enumerate(input_indices):
+            call_args[inp] = spec_data[j]
+        new_call = spec_call(*call_args)
+
+        # Remove meaningless padded outputs if applicable.
+        if auto_pad and affects_output:
+            new_call = relay.take(
+                new_call,
+                relay.arange(start=relay.const(0), stop=flex_value, dtype="int32"),
+                axis=dim,
+            )
+
+        # Add this new case to the dispatch handler.
+        if_exprs.append((relay.op.equal(check_dim, relay.const(bucket)), new_call))
+
+    # Create a subgraph to handle all other shapes.
+    default_dyn_call, _ = specialize_body(
+        mod, main_fn, dim, relay.Any(), input_indices=input_indices, affects_output=affects_output
+    )
+    call_args = list(main_fn.params)
+    for j, inp in enumerate(input_indices):
+        call_args[inp] = dyn_data[j]
+    new_body = default_dyn_call(*call_args)
+
+    # Create an If chain to dispatch shapes to the appropriate specialized subgraph.
+    for cond, true_branch in if_exprs:
+        new_body = relay.If(cond, true_branch, new_body)
+
+    # Assign new parameters to the function.
+    new_params = list(main_fn.params)
+    for j, inp in enumerate(input_indices):
+        new_params[inp] = dyn_data[j]
+
+    # Update the output shape to be dynamic if needed.
+    if affects_output:
+        dyn_ret_type = override_shape(main_fn.ret_type, dim, relay.Any())
+    else:
+        dyn_ret_type = main_fn.ret_type
+
+    # Assign the handler as the new entrypoint in the module.
+    new_main = relay.Function(
+        new_params, new_body, dyn_ret_type, main_fn.type_params, main_fn.attrs
+    )
+    mod["main"] = new_main
+    return mod
+
+
+class FlexibleShapeDispatch(object):
+    """Enable inference of multiple shaped inputs in one module.
+
+    This transformation adds a handler around a module that
+    checks input shapes and dispatches to a subgraph specialized
+    to handle the specific shapes of that input. If no exactly matching
+    subgraph is available, the input will be run using full dynamism.
+    For best performance, specify all the sizes the module will
+    be likely to see using the buckets argument.
+
+    Parameters
+    ----------
+    dim: int
+        The dimension of the input that should be made flexible. This will
+        most often be used for the batch dimension.
+    buckets: list[int]
+        The sizes of the input dimension that should be explicitly handled.
+        Each value in buckets will have a corresponding subgraph constructed to
+        handle it.
+    auto_pad: Optional[bool]
+        If True, then padding will be inserted to values that don't match one of
+        the provided buckets.
+    pad_value: Optional[float]
+        When auto_pad is true, padding will be done with this value.
+    input_indices: Optional[List[int]]
+        Which inputs should be dispatched dynamically, provided by index. All inputs
+        must share the same dynamic axis.
+    affects_output: Optional[bool]
+        Whether the change in input shape has a corresponding effect on the output shape.
+        Batching for example effects both the input and output whereas changing sequence
+        length in an NLP model typically does not.
+
+    Returns
+    -------
+    ret : FlexibleShapeDispatch
+        A pass that can be applied to a module to add flexible shape handling.
+    """
+
+    def __init__(
+        self,
+        dim=0,
+        buckets=[1],

Review Comment:
   same as above. 



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

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 diff in pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jroesch commented on code in PR #11199:
URL: https://github.com/apache/tvm/pull/11199#discussion_r863737070


##########
python/tvm/relay/transform/flexible_shape.py:
##########
@@ -0,0 +1,273 @@
+# 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.
+# pylint: disable=invalid-name, dangerous-default-value
+"""Relay functions for wrapping a module with flexible shape dispatch."""
+from tvm import relay
+
+
+def override_shape(tensor_type, dim, value):
+    """Change a value in a tensor shape."""
+    new_dims = list(tensor_type.shape)
+    new_dims[dim] = value
+    return relay.TensorType(new_dims, tensor_type.dtype)
+
+
+def specialize_body(mod, function, dim, value, input_indices=[0], affects_output=True):
+    """Create a subgraph to handle specific input shapes"""
+    # Iterate through specified inputs and construct specialized shapes for each.
+    new_params = list(function.params)
+    data_binding = {}
+    dyn_data_array = []
+    for inp in input_indices:
+        data = function.params[inp]
+        flex_ty = override_shape(data.type_annotation, dim, value)
+        dyn_data = relay.Var(data.name_hint, type_annotation=flex_ty)
+        new_params[inp] = dyn_data
+        data_binding[data] = dyn_data
+        dyn_data_array.append(dyn_data)
+
+    # Create a new function body for the modified shapes.
+    new_body = relay.expr.bind(function.body, data_binding)
+    # Only change the output shape if the input shape affects it.
+    if affects_output:
+        new_ret_ty = override_shape(function.ret_type, dim, value)
+    else:
+        new_ret_ty = function.ret_type
+    gvar = relay.GlobalVar("main_" + str(value))
+    # Add the new function to the main IRModule.
+    mod[gvar] = relay.Function(
+        new_params, new_body, new_ret_ty, function.type_params, function.attrs
+    )
+    return gvar, [d.type_annotation for d in dyn_data_array]
+
+
+def flexible_dispatch(
+    mod, dim=0, buckets=[1], auto_pad=False, pad_value=0, input_indices=[0], affects_output=True

Review Comment:
   Same comment as above, probably worth specifying the modes in the doc string if possible to enumerate. 



##########
python/tvm/relay/transform/flexible_shape.py:
##########
@@ -0,0 +1,273 @@
+# 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.
+# pylint: disable=invalid-name, dangerous-default-value
+"""Relay functions for wrapping a module with flexible shape dispatch."""
+from tvm import relay
+
+
+def override_shape(tensor_type, dim, value):
+    """Change a value in a tensor shape."""
+    new_dims = list(tensor_type.shape)
+    new_dims[dim] = value
+    return relay.TensorType(new_dims, tensor_type.dtype)
+
+
+def specialize_body(mod, function, dim, value, input_indices=[0], affects_output=True):
+    """Create a subgraph to handle specific input shapes"""
+    # Iterate through specified inputs and construct specialized shapes for each.
+    new_params = list(function.params)
+    data_binding = {}
+    dyn_data_array = []
+    for inp in input_indices:
+        data = function.params[inp]
+        flex_ty = override_shape(data.type_annotation, dim, value)
+        dyn_data = relay.Var(data.name_hint, type_annotation=flex_ty)
+        new_params[inp] = dyn_data
+        data_binding[data] = dyn_data
+        dyn_data_array.append(dyn_data)
+
+    # Create a new function body for the modified shapes.
+    new_body = relay.expr.bind(function.body, data_binding)
+    # Only change the output shape if the input shape affects it.
+    if affects_output:
+        new_ret_ty = override_shape(function.ret_type, dim, value)
+    else:
+        new_ret_ty = function.ret_type
+    gvar = relay.GlobalVar("main_" + str(value))
+    # Add the new function to the main IRModule.
+    mod[gvar] = relay.Function(
+        new_params, new_body, new_ret_ty, function.type_params, function.attrs
+    )
+    return gvar, [d.type_annotation for d in dyn_data_array]
+
+
+def flexible_dispatch(
+    mod, dim=0, buckets=[1], auto_pad=False, pad_value=0, input_indices=[0], affects_output=True

Review Comment:
   Same comment as above about defaults, probably worth specifying the modes in the doc string if possible to enumerate. 



-- 
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 diff in pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jroesch commented on code in PR #11199:
URL: https://github.com/apache/tvm/pull/11199#discussion_r863733535


##########
python/tvm/relay/transform/flexible_shape.py:
##########
@@ -0,0 +1,273 @@
+# 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.
+# pylint: disable=invalid-name, dangerous-default-value
+"""Relay functions for wrapping a module with flexible shape dispatch."""
+from tvm import relay
+
+
+def override_shape(tensor_type, dim, value):
+    """Change a value in a tensor shape."""

Review Comment:
   ```suggestion
       """Change a dimension in a tensor shape."""
   ```



-- 
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] jwfromm merged pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jwfromm merged PR #11199:
URL: https://github.com/apache/tvm/pull/11199


-- 
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 diff in pull request #11199: [Relay] Flexible shape dispatch transformation

Posted by GitBox <gi...@apache.org>.
jroesch commented on code in PR #11199:
URL: https://github.com/apache/tvm/pull/11199#discussion_r863734591


##########
python/tvm/relay/transform/flexible_shape.py:
##########
@@ -0,0 +1,273 @@
+# 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.
+# pylint: disable=invalid-name, dangerous-default-value
+"""Relay functions for wrapping a module with flexible shape dispatch."""
+from tvm import relay
+
+
+def override_shape(tensor_type, dim, value):
+    """Change a value in a tensor shape."""
+    new_dims = list(tensor_type.shape)
+    new_dims[dim] = value
+    return relay.TensorType(new_dims, tensor_type.dtype)
+
+
+def specialize_body(mod, function, dim, value, input_indices=[0], affects_output=True):
+    """Create a subgraph to handle specific input shapes"""

Review Comment:
   ```suggestion
       """Specialize a function to specific input shapes. This function has a few modes XXX"""
   ```
   
   Would be worth explaining this a little more. 



-- 
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 #11199: [Relay] Flexible shape dispatch transformation

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

   cc @mbs-octoml 


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