data-tiling: introduce `upper_bound_tile_size` op to defer padding-size choice to MaterializeEncoding. (#14349)

This fixes #11632, by introducing a materializable
`upper_bound_tile_size ` instead of hardcoding a fixed padding amount at
Flow, and fixes it in sufficient generality to also solve the problem
for narrow matmuls - let's explain that in more detail as this is an
important part of what this PR is doing.

For each combination of element types and each target, the
MaterializeEncoding pass selects appropriate matmul tile shapes. Input
tensors get padded to the next multiple of the next tile size. The
padding increases the inherent arithmetic cost of the problem at hand.
When, along some dimension, the original tensor size is smaller than the
tile size, that can result in particulary large overhead. The extreme
case, which is also a very common case, is matrix-times-vector
multiplication. The "vector" right-hand side is really a matrix with one
dimension size equal to 1, so if the general matmul tile shape along
that dimension is 8 or 16, as is usually the case, that can be a 8x or
16x increase in the inherent arithmetic cost of the matmul op.

The solution to that is to adjust MaterializeEncoding tile shapes to
narrow dimensions. We had some logic in place to deal with that, but
#11632 was leaving it moot: the flow-level padding of everything to the
next multiple of 16 meant that our logic there never really had a chance
of kicking in. With #11632 being fixed, this PR was the opportunity to
also fix that along the way, and to ensure that the solution to #11632
worked also in that respect. As matrix-times-vector products were the
common case that suffered the most from #11632, it would have been too
bad to "solve" #11632 without addressing that. By the way,
matrix-times-vector is only the extreme case, but other narrow cases
matter too. When, e.g. on AVX-512, the general matmul tile size is 16,
even width-8 matmuls (MxKx8) were suffering from 2x-widening. So the
solution in this PR is making sure to address all narrow cases, defined
as whenever a tensor dimension size is less than the general tile size.

The difficulty was that when MaterializeEncoding runs on a dispatch
function, it runs on an already-padded tensor; even as this PR
introduces `upper_bound_tile_size`, that only makes it possible to
select the right padding amount, but there's still a `tensor.pad` op and
it's still getting in the way of knowing the actual, original tensor
shape for the purpose of adjusting tile shapes for narrow cases.
Moreover, as `MaterializeEncoding` is a type-converter pass, it can't
just walk from a Value up to its defining-op to find the pre-padding
tensor. There are no values there, only types. So the information about
the pre-padding tensor shape has to be part of the tensor type that
`MaterializeEncoding` sees, that its, the padded tensor type.

The solution to that problem in this PR is to add a `original_type`
field to `EncodingAttr`.

Fixes  #11632.

Fixes a compiler issue encountered in #14398 but not the originally
reported runtime crash by itself.

This now also includes the removal of a now-useless VMVX pass, which was
originally split into https://github.com/openxla/iree/pull/14383 .
diff --git a/compiler/src/iree/compiler/Codegen/BUILD.bazel b/compiler/src/iree/compiler/Codegen/BUILD.bazel
index 5d6a3fc..a2c7bdc 100644
--- a/compiler/src/iree/compiler/Codegen/BUILD.bazel
+++ b/compiler/src/iree/compiler/Codegen/BUILD.bazel
@@ -22,6 +22,7 @@
     ],
     deps = [
         "//compiler/src/iree/compiler/Codegen/Common",
+        "//compiler/src/iree/compiler/Codegen/Common/CPU:CommonCPUPasses",
         "//compiler/src/iree/compiler/Codegen/Common/GPU:CommonGPUPasses",
         "//compiler/src/iree/compiler/Codegen/Dialect:IREECodegenDialect",
         "//compiler/src/iree/compiler/Codegen/LLVMCPU",
diff --git a/compiler/src/iree/compiler/Codegen/CMakeLists.txt b/compiler/src/iree/compiler/Codegen/CMakeLists.txt
index 40dc173..26ea957 100644
--- a/compiler/src/iree/compiler/Codegen/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Codegen/CMakeLists.txt
@@ -21,6 +21,7 @@
     IREELinalgExtPasses
     MLIRPass
     iree::compiler::Codegen::Common
+    iree::compiler::Codegen::Common::CPU::CommonCPUPasses
     iree::compiler::Codegen::Common::GPU::CommonGPUPasses
     iree::compiler::Codegen::Dialect::IREECodegenDialect
     iree::compiler::Codegen::LLVMCPU
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/BUILD.bazel b/compiler/src/iree/compiler/Codegen/Common/CPU/BUILD.bazel
new file mode 100644
index 0000000..88aea51
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/BUILD.bazel
@@ -0,0 +1,90 @@
+# Copyright 2023 The IREE Authors
+#
+# Licensed under the Apache License v2.0 with LLVM Exceptions.
+# See https://llvm.org/LICENSE.txt for license information.
+# SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+load("//build_tools/bazel:build_defs.oss.bzl", "iree_compiler_cc_library", "iree_gentbl_cc_library")
+
+package(
+    default_visibility = ["//visibility:public"],
+    features = ["layering_check"],
+    licenses = ["notice"],  # Apache 2.0
+)
+
+iree_gentbl_cc_library(
+    name = "PassesIncGen",
+    tbl_outs = [
+        (
+            ["--gen-pass-decls"],
+            "Passes.h.inc",
+        ),
+    ],
+    tblgen = "@llvm-project//mlir:mlir-tblgen",
+    td_file = "Passes.td",
+    deps = ["@llvm-project//mlir:PassBaseTdFiles"],
+)
+
+iree_compiler_cc_library(
+    name = "PassHeaders",
+    hdrs = [
+        "PassDetail.h",
+        "Passes.h",
+        "Passes.h.inc",
+    ],
+    deps = [
+        ":PassesIncGen",
+        "//compiler/src/iree/compiler/Codegen/Dialect:IREECodegenDialect",
+        "//compiler/src/iree/compiler/Dialect/HAL/IR",
+        "//compiler/src/iree/compiler/Utils",
+        "@llvm-project//mlir:Pass",
+        "@llvm-project//mlir:Transforms",
+    ],
+)
+
+iree_compiler_cc_library(
+    name = "CommonCPUPasses",
+    srcs = [
+        "CPUMaterializeEncodingPass.cpp",
+        "Passes.cpp",
+    ],
+    hdrs = [
+        "Passes.h",
+    ],
+    deps = [
+        ":PassHeaders",
+        ":PassesIncGen",
+        "//compiler/src/iree/compiler/Codegen/Common",
+        "//compiler/src/iree/compiler/Codegen/Dialect:IREECodegenDialect",
+        "//compiler/src/iree/compiler/Codegen/Transforms",
+        "//compiler/src/iree/compiler/Codegen/Utils",
+        "//compiler/src/iree/compiler/Dialect/HAL/IR",
+        "//llvm-external-projects/iree-dialects:IREELinalgExtDialect",
+        "//llvm-external-projects/iree-dialects:IREELinalgExtTransforms",
+        "//llvm-external-projects/iree-dialects:IREELinalgExtUtils",
+        "@llvm-project//llvm:Support",
+        "@llvm-project//mlir:AffineDialect",
+        "@llvm-project//mlir:AffineTransforms",
+        "@llvm-project//mlir:AffineUtils",
+        "@llvm-project//mlir:ArithDialect",
+        "@llvm-project//mlir:BufferizationDialect",
+        "@llvm-project//mlir:DestinationStyleOpInterface",
+        "@llvm-project//mlir:FuncDialect",
+        "@llvm-project//mlir:IR",
+        "@llvm-project//mlir:LinalgDialect",
+        "@llvm-project//mlir:LinalgUtils",
+        "@llvm-project//mlir:MemRefDialect",
+        "@llvm-project//mlir:MemRefTransforms",
+        "@llvm-project//mlir:Pass",
+        "@llvm-project//mlir:SCFDialect",
+        "@llvm-project//mlir:SCFTransforms",
+        "@llvm-project//mlir:SCFUtils",
+        "@llvm-project//mlir:SideEffectInterfaces",
+        "@llvm-project//mlir:Support",
+        "@llvm-project//mlir:TensorDialect",
+        "@llvm-project//mlir:Transforms",
+        "@llvm-project//mlir:VectorDialect",
+        "@llvm-project//mlir:VectorToSCF",
+        "@llvm-project//mlir:VectorTransforms",
+    ],
+)
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/CMakeLists.txt b/compiler/src/iree/compiler/Codegen/Common/CPU/CMakeLists.txt
new file mode 100644
index 0000000..d1ed3f3
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/CMakeLists.txt
@@ -0,0 +1,85 @@
+################################################################################
+# Autogenerated by build_tools/bazel_to_cmake/bazel_to_cmake.py from           #
+# compiler/src/iree/compiler/Codegen/Common/CPU/BUILD.bazel                    #
+#                                                                              #
+# Use iree_cmake_extra_content from iree/build_defs.oss.bzl to add arbitrary   #
+# CMake-only content.                                                          #
+#                                                                              #
+# To disable autogeneration for this file entirely, delete this header.        #
+################################################################################
+
+iree_add_all_subdirs()
+
+iree_tablegen_library(
+  NAME
+    PassesIncGen
+  TD_FILE
+    "Passes.td"
+  OUTS
+    --gen-pass-decls Passes.h.inc
+)
+
+iree_cc_library(
+  NAME
+    PassHeaders
+  HDRS
+    "PassDetail.h"
+    "Passes.h"
+    "Passes.h.inc"
+  DEPS
+    ::PassesIncGen
+    MLIRPass
+    MLIRTransforms
+    iree::compiler::Codegen::Dialect::IREECodegenDialect
+    iree::compiler::Dialect::HAL::IR
+    iree::compiler::Utils
+  PUBLIC
+)
+
+iree_cc_library(
+  NAME
+    CommonCPUPasses
+  HDRS
+    "Passes.h"
+  SRCS
+    "CPUMaterializeEncodingPass.cpp"
+    "Passes.cpp"
+  DEPS
+    ::PassHeaders
+    ::PassesIncGen
+    IREELinalgExtDialect
+    IREELinalgExtTransforms
+    IREELinalgExtUtils
+    LLVMSupport
+    MLIRAffineDialect
+    MLIRAffineTransforms
+    MLIRAffineUtils
+    MLIRArithDialect
+    MLIRBufferizationDialect
+    MLIRDestinationStyleOpInterface
+    MLIRFuncDialect
+    MLIRIR
+    MLIRLinalgDialect
+    MLIRLinalgUtils
+    MLIRMemRefDialect
+    MLIRMemRefTransforms
+    MLIRPass
+    MLIRSCFDialect
+    MLIRSCFTransforms
+    MLIRSCFUtils
+    MLIRSideEffectInterfaces
+    MLIRSupport
+    MLIRTensorDialect
+    MLIRTransforms
+    MLIRVectorDialect
+    MLIRVectorToSCF
+    MLIRVectorTransforms
+    iree::compiler::Codegen::Common
+    iree::compiler::Codegen::Dialect::IREECodegenDialect
+    iree::compiler::Codegen::Transforms
+    iree::compiler::Codegen::Utils
+    iree::compiler::Dialect::HAL::IR
+  PUBLIC
+)
+
+### BAZEL_TO_CMAKE_PRESERVES_ALL_CONTENT_BELOW_THIS_LINE ###
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/CPUMaterializeEncodingPass.cpp b/compiler/src/iree/compiler/Codegen/Common/CPU/CPUMaterializeEncodingPass.cpp
new file mode 100644
index 0000000..c270a8c
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/CPUMaterializeEncodingPass.cpp
@@ -0,0 +1,300 @@
+// Copyright 2023 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#include "iree-dialects/Dialect/LinalgExt/IR/LinalgExtDialect.h"
+#include "iree-dialects/Dialect/LinalgExt/IR/LinalgExtOps.h"
+#include "iree-dialects/Dialect/LinalgExt/Passes/Passes.h"
+#include "iree-dialects/Dialect/LinalgExt/Utils/Utils.h"
+#include "iree/compiler/Codegen/Common/CPU/PassDetail.h"
+#include "iree/compiler/Codegen/Common/CPU/Passes.h"
+#include "iree/compiler/Codegen/Common/EncodingInfo.h"
+#include "iree/compiler/Codegen/Dialect/IREECodegenDialect.h"
+#include "iree/compiler/Codegen/Utils/Utils.h"
+#include "iree/compiler/Dialect/HAL/IR/HALTypes.h"
+#include "mlir/Dialect/Arith/IR/Arith.h"
+#include "mlir/Dialect/MemRef/Transforms/Transforms.h"
+#include "mlir/IR/BuiltinTypeInterfaces.h"
+#include "mlir/IR/BuiltinTypes.h"
+#include "mlir/Transforms/DialectConversion.h"
+#include "mlir/Transforms/GreedyPatternRewriteDriver.h"
+
+namespace mlir {
+namespace iree_compiler {
+
+using namespace IREE::LinalgExt;
+using IREE::HAL::ExecutableTargetAttr;
+
+namespace {
+
+static MatmulTileParams
+chooseMatmulTileParamsGeneric(ExecutableTargetAttr target) {
+  if (isVMVXBackend(target) && hasMicrokernels(target)) {
+    // VMVX+ukernel uses dynamic tile shapes.
+    return {ShapedType::kDynamic, ShapedType::kDynamic, ShapedType::kDynamic};
+  } else {
+    // Some vaguely reasonable static tile shape.
+    return {8, 4, 8};
+  }
+}
+
+static MatmulTileParams
+chooseMatmulTileParamsAArch64(EncodingUser user, ExecutableTargetAttr target) {
+  switch (user) {
+  case EncodingUser::MATMUL_F32F32F32:
+  case EncodingUser::MATMUL_F16F16F32:
+  case EncodingUser::MATMUL_F16F16F16:
+  case EncodingUser::MATMUL_BF16BF16F32:
+  case EncodingUser::MATMUL_BF16BF16BF16:
+    // Note: 16-bit floating point types currently use the same tile size as
+    // f32. This makes sense when either (1) the accumulator is f32, or (2)
+    // the arithmetic will have to expand f16 to f32 in registers. We may
+    // reconsider when taking advantage of native f16/bf16 arithmetic when the
+    // accumulator itself is f16/bf16.
+    return {8, 1, 8};
+  case EncodingUser::MATMUL_I8I8I32:
+    if (hasFeature(target, "+i8mm")) {
+      // Aim to use SMMLA.
+      return {8, 8, 8};
+    }
+    if (hasFeature(target, "+dotprod")) {
+      // Aim to use SDOT.
+      return {8, 4, 8};
+    }
+    return {8, 1, 8};
+  default:
+    assert(false);
+    return {};
+  }
+}
+
+static MatmulTileParams
+chooseMatmulTileParamsX86_64(EncodingUser user, ExecutableTargetAttr target) {
+  switch (user) {
+  case EncodingUser::MATMUL_F32F32F32:
+  case EncodingUser::MATMUL_F16F16F32:
+  case EncodingUser::MATMUL_F16F16F16:
+  case EncodingUser::MATMUL_BF16BF16F32:
+  case EncodingUser::MATMUL_BF16BF16BF16:
+    // Note: 16-bit floating point types currently use the same tile size as
+    // f32. This makes sense when either (1) the accumulator is f32, or (2)
+    // the arithmetic will have to expand f16 to f32 in registers. We may
+    // reconsider when taking advantage of native f16/bf16 arithmetic when the
+    // accumulator itself is f16/bf16.
+    if (hasFeature(target, "+avx512f")) {
+      return {16, 1, 16};
+    }
+    if (hasFeature(target, "+avx")) {
+      // Note: for good performance, most +avx users will also want to add
+      // +fma, but that's a local instruction selection detail and the tile
+      // layout is unaffected, as there are enough registers even with the
+      // need for intermediate product registers when +fma is not used.
+      return {8, 1, 8};
+    }
+    // SSE fallback.
+    return {8, 1, 4};
+  case EncodingUser::MATMUL_I8I8I32:
+    if (hasFeature(target, "+avx512vnni")) {
+      // Aim to use VPDPWSSD. This is the same tile size as with VPMADDWD
+      // as the only difference is that VPDPWSSD accumulates. VPDPBUSD would
+      // call for {16, 4, 16} but we can't use it because of its unsigned LHS.
+      return {16, 2, 16};
+    }
+    if (hasFeature(target, "+avx512bw")) {
+      // Aim to use VPMADDWD (zmm).
+      return {16, 2, 16};
+    }
+    if (hasFeature(target, "+avx2")) {
+      // Aim to use VPMADDWD (ymm).
+      return {8, 2, 8};
+    }
+    // SSE fallback. Aim to use PMADDWD (xmm).
+    return {8, 2, 4};
+  default:
+    assert(false);
+    return {};
+  }
+}
+
+static MatmulTileParams chooseMatmulTileParams(EncodingUser user,
+                                               ExecutableTargetAttr target) {
+  if (isAArch64(target)) {
+    return chooseMatmulTileParamsAArch64(user, target);
+  }
+  if (isX86_64(target)) {
+    return chooseMatmulTileParamsX86_64(user, target);
+  }
+  return chooseMatmulTileParamsGeneric(target);
+}
+
+struct CPUMaterializeEncodingPass
+    : public CPUMaterializeEncodingBase<CPUMaterializeEncodingPass> {
+  void getDependentDialects(DialectRegistry &registry) const override {
+    registry.insert<arith::ArithDialect, IREE::LinalgExt::IREELinalgExtDialect,
+                    IREE::Codegen::IREECodegenDialect>();
+  }
+  void runOnOperation() override;
+};
+
+struct CPUMaterializeUpperBoundTileSizePass
+    : public CPUMaterializeUpperBoundTileSizeBase<
+          CPUMaterializeUpperBoundTileSizePass> {
+  void getDependentDialects(DialectRegistry &registry) const override {
+    registry.insert<arith::ArithDialect>();
+  }
+  void runOnOperation() override;
+};
+
+FailureOr<MaterializeEncodingInfo>
+materializeEncodingForTarget(RankedTensorType tensorType,
+                             ExecutableTargetAttr targetAttr) {
+  IREE::LinalgExt::EncodingAttr encoding =
+      tensorType.getEncoding()
+          .dyn_cast_or_null<IREE::LinalgExt::EncodingAttr>();
+  if (!encoding) {
+    return failure();
+  }
+  auto user = encoding.getUser().getValue();
+  auto role = encoding.getRole().getValue();
+  MatmulTileParams tileParams = chooseMatmulTileParams(user, targetAttr);
+  auto encodingInfo = chooseEncodingInfoForMatmul(role, tileParams);
+  auto originalTypeAttr = encoding.getOriginalType();
+  auto originalType = originalTypeAttr
+                          ? originalTypeAttr.getValue().cast<RankedTensorType>()
+                          : tensorType;
+  // TODO(bjacob): not sure why this causes buffer issues with VMVX.
+  if (!isVMVXBackend(targetAttr)) {
+    adjustTileSizesToNarrowStaticShape(encodingInfo, originalType.getShape());
+  }
+  return encodingInfo;
+}
+
+MaterializeEncodingFn
+getMaterializeEncodingFn(ExecutableTargetAttr targetAttr) {
+  return
+      [targetAttr](
+          RankedTensorType tensorType) -> FailureOr<MaterializeEncodingInfo> {
+        return materializeEncodingForTarget(tensorType, targetAttr);
+      };
+}
+
+// Like getMaterializeEncodingFn, but iterating over an array of targets and
+// returning the max of all tile sizes from each target, checking that other
+// materialization info (permutations) agree.
+//
+// This is useful to compute padding amounts, in the materialization of
+// UpperBoundTileSizeOp, in top-level functions that are not part of one HAL
+// executable variant. There, the padding amounts only control the size of
+// allocated buffers, so it's OK to over-estimate (only wasting some memory)
+// but not under-estimate (would cause buffer overruns) padding amounts.
+MaterializeEncodingFn
+getUpperBoundMaterializeEncodingFn(ArrayRef<ExecutableTargetAttr> targetAttrs) {
+  return
+      [targetAttrs](
+          RankedTensorType tensorType) -> FailureOr<MaterializeEncodingInfo> {
+        FailureOr<MaterializeEncodingInfo> result; // Defaults to failure.
+        for (auto targetAttr : targetAttrs) {
+          FailureOr<MaterializeEncodingInfo> info =
+              materializeEncodingForTarget(tensorType, targetAttr);
+          if (failed(info)) {
+            // No info at this iteration. Ignore and continue.
+            continue;
+          }
+          if (failed(result)) {
+            // No preexisting result. Use this iteration's info and continue.
+            result = info;
+            continue;
+          }
+          // Merge this iteration's info into preexisting result info.
+          // Check that permutations match, then record the max of tile sizes.
+          if (info->innerDimsPos != result->innerDimsPos ||
+              info->outerDimsPerm != result->outerDimsPerm) {
+            return failure();
+          }
+          if (info->innerTileSizes.size() != result->innerTileSizes.size()) {
+            return failure();
+          }
+          for (unsigned i = 0; i < info->innerTileSizes.size(); ++i) {
+            if (info->innerTileSizes[i] == ShapedType::kDynamic) {
+              result->innerTileSizes[i] = ShapedType::kDynamic;
+            } else {
+              result->innerTileSizes[i] =
+                  std::max(result->innerTileSizes[i], info->innerTileSizes[i]);
+            }
+          }
+        }
+        return result;
+      };
+}
+
+} // namespace
+
+void CPUMaterializeEncodingPass::runOnOperation() {
+  MLIRContext *context = &getContext();
+  auto operation = getOperation();
+  RewritePatternSet materializeEncodingPattern(context);
+  auto targetAttr = ExecutableTargetAttr::lookup(operation);
+  auto materializeEncodingFn = getMaterializeEncodingFn(targetAttr);
+  if (!materializeEncodingFn) {
+    return signalPassFailure();
+  }
+  MaterializeEncodingTypeConverter typeConverter(materializeEncodingFn);
+  MaterializeEncodingConversionTarget target(*context);
+  auto materializeEncodingValueFn = getMaterializeEncodingValueFn(targetAttr);
+  populateMaterializeEncodingIntoPackUnPackPatterns(materializeEncodingPattern,
+                                                    target, typeConverter,
+                                                    materializeEncodingValueFn);
+
+  if (failed(applyPartialConversion(operation, target,
+                                    std::move(materializeEncodingPattern)))) {
+    operation.emitOpError("materialization failed");
+    return signalPassFailure();
+  }
+
+  // Add patterns to fold pack/unpack ops with pad/extract_slice ops and resolve
+  // dims ops.
+  {
+    RewritePatternSet patterns(context);
+    tensor::populateFoldIntoPackAndUnpackPatterns(patterns);
+    memref::populateResolveRankedShapedTypeResultDimsPatterns(patterns);
+    if (failed(applyPatternsAndFoldGreedily(operation, std::move(patterns)))) {
+      operation.emitOpError("folding patterns failed");
+      return signalPassFailure();
+    }
+  }
+}
+
+void CPUMaterializeUpperBoundTileSizePass::runOnOperation() {
+  MLIRContext *context = &getContext();
+  auto operation = getOperation();
+  auto targetAttrs =
+      IREE::HAL::DeviceTargetAttr::lookupExecutableTargets(operation);
+  RewritePatternSet patterns(context);
+  MaterializeEncodingFn materializeEncodingFn =
+      getUpperBoundMaterializeEncodingFn(targetAttrs);
+  if (!materializeEncodingFn) {
+    return signalPassFailure();
+  }
+  populateMaterializeUpperBoundTileSizePatterns(patterns,
+                                                materializeEncodingFn);
+  if (failed(applyPatternsAndFoldGreedily(operation, std::move(patterns)))) {
+    operation.emitOpError(
+        "encoding padding sizes materialization pattern failed");
+    return signalPassFailure();
+  }
+}
+
+std::unique_ptr<OperationPass<func::FuncOp>>
+createCPUMaterializeEncodingPass() {
+  return std::make_unique<CPUMaterializeEncodingPass>();
+}
+
+std::unique_ptr<InterfacePass<mlir::FunctionOpInterface>>
+createCPUMaterializeUpperBoundTileSizePass() {
+  return std::make_unique<CPUMaterializeUpperBoundTileSizePass>();
+}
+
+} // namespace iree_compiler
+} // namespace mlir
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/PassDetail.h b/compiler/src/iree/compiler/Codegen/Common/CPU/PassDetail.h
new file mode 100644
index 0000000..cfc4a9f
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/PassDetail.h
@@ -0,0 +1,21 @@
+// Copyright 2023 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#ifndef IREE_COMPILER_CODEGEN_LLVMCPU_PASS_DETAIL_H_
+#define IREE_COMPILER_CODEGEN_LLVMCPU_PASS_DETAIL_H_
+
+#include "mlir/Pass/Pass.h"
+
+namespace mlir {
+namespace iree_compiler {
+
+#define GEN_PASS_CLASSES
+#include "iree/compiler/Codegen/Common/CPU/Passes.h.inc"
+
+} // namespace iree_compiler
+} // namespace mlir
+
+#endif // IREE_COMPILER_CODEGEN_LLVMCPU_PASS_DETAIL_H_
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.cpp b/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.cpp
new file mode 100644
index 0000000..1f2b71f
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.cpp
@@ -0,0 +1,27 @@
+// Copyright 2023 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#include "iree/compiler/Codegen/Common/CPU/Passes.h"
+#include "mlir/Pass/PassManager.h"
+
+namespace mlir {
+namespace iree_compiler {
+
+//===---------------------------------------------------------------------===//
+// Register Common/CPU Passes
+//===---------------------------------------------------------------------===//
+
+namespace {
+#define GEN_PASS_REGISTRATION
+#include "iree/compiler/Codegen/Common/CPU/Passes.h.inc"
+} // namespace
+
+void registerCodegenCommonCPUPasses() {
+  // Generated.
+  registerPasses();
+}
+} // namespace iree_compiler
+} // namespace mlir
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.h b/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.h
new file mode 100644
index 0000000..0df9d03
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.h
@@ -0,0 +1,50 @@
+// Copyright 2023 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+//===----------------------------------------------------------------------===//
+//
+// This file includes the LLVMCPU Passes.
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef IREE_COMPILER_CODEGEN_COMMON_CPU_PASSES_H_
+#define IREE_COMPILER_CODEGEN_COMMON_CPU_PASSES_H_
+
+#include "mlir/Dialect/Func/IR/FuncOps.h"
+#include "mlir/Pass/Pass.h"
+
+namespace mlir {
+namespace iree_compiler {
+
+/// Convert encoding-specific operations based on target attributes. Examples:
+///   linalg_ext.set_encoding   -> tensor.pack
+///   linalg_ext.unset_encoding -> tensor.unpack
+///   linalg.matmul             -> linalg.mmt4d
+std::unique_ptr<OperationPass<func::FuncOp>> createCPUMaterializeEncodingPass();
+
+/// Like createLLVMCPUMaterializeEncodingPass, but specifically for
+/// linalg_ext.upper_bound_tile_size, converting it to constants.
+///
+/// Unlike createLLVMCPUMaterializeEncodingPass, this does not require the
+/// op to have a specific HAL target attribute. Instead, this will iterate over
+/// all HAL target attributes, use the maximum of all padding sizes from each
+/// target. This is needed because in top-level functions outside of HAL
+/// executables, there are upper_bound_tile_size ops (created by SetEncoding,
+/// and computing buffer allocation sizes) and there isn't one specific HAL
+/// target.
+///
+/// In the VMVX case where padding sizes are not compile-time constants, this
+/// converts upper_bound_tile_size to some specific constant size (currently 16)
+/// that is the largest tile size that we can use in VMVX, and can be adjusted
+// as needed.
+std::unique_ptr<InterfacePass<mlir::FunctionOpInterface>>
+createCPUMaterializeUpperBoundTileSizePass();
+
+void registerCodegenCommonCPUPasses();
+
+} // namespace iree_compiler
+} // namespace mlir
+
+#endif // IREE_COMPILER_CODEGEN_COMMON_CPU_PASSES_H_
diff --git a/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.td b/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.td
new file mode 100644
index 0000000..f1cc23c
--- /dev/null
+++ b/compiler/src/iree/compiler/Codegen/Common/CPU/Passes.td
@@ -0,0 +1,28 @@
+// Copyright 2023 The IREE Authors
+//
+// Licensed under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+
+#ifndef IREE_CODEGEN_COMMON_CPU_PASSES
+#define IREE_CODEGEN_COMMON_CPU_PASSES
+
+include "mlir/Pass/PassBase.td"
+
+//===---------------------------------------------------------------------===//
+// Common Passes used for CPU-like backends (keep alphabetical)
+//===---------------------------------------------------------------------===//
+
+def CPUMaterializeEncoding :
+    Pass<"iree-cpu-materialize-encoding", "func::FuncOp"> {
+  let summary = "Materialize the encoding for tensor as specified by the backend";
+  let constructor = "mlir::iree_compiler::createCPUMaterializeEncodingPass()";
+}
+
+def CPUMaterializeUpperBoundTileSize :
+    InterfacePass<"iree-cpu-materialize-upper-bound-tile-size", "mlir::FunctionOpInterface"> {
+  let summary = "Materialize upper_bound_tile_size to constants.";
+  let constructor = "mlir::iree_compiler::createCPUMaterializeUpperBoundTileSizePass()";
+}
+
+#endif  // IREE_CODEGEN_COMMON_CPU_PASSES
diff --git a/compiler/src/iree/compiler/Codegen/Common/DecomposePackUnPackOps.cpp b/compiler/src/iree/compiler/Codegen/Common/DecomposePackUnPackOps.cpp
index 01852af..771b73b 100644
--- a/compiler/src/iree/compiler/Codegen/Common/DecomposePackUnPackOps.cpp
+++ b/compiler/src/iree/compiler/Codegen/Common/DecomposePackUnPackOps.cpp
@@ -85,6 +85,12 @@
     if (numDropDims == 0)
       return failure();
 
+    // Dropping all dims. Elide now to avoid corner cases.
+    if (numDropDims == inputTy.getRank()) {
+      rewriter.replaceOp(op, op.getInput());
+      return success();
+    }
+
     Location loc = op.getLoc();
     SmallVector<OpFoldResult> srcMixedSizes =
         tensor::getMixedSizes(rewriter, loc, op.getInput());
diff --git a/compiler/src/iree/compiler/Codegen/Common/EncodingInfo.h b/compiler/src/iree/compiler/Codegen/Common/EncodingInfo.h
index c0c2524..9641529 100644
--- a/compiler/src/iree/compiler/Codegen/Common/EncodingInfo.h
+++ b/compiler/src/iree/compiler/Codegen/Common/EncodingInfo.h
@@ -37,14 +37,6 @@
     IREE::LinalgExt::MaterializeEncodingTypeConverter &typeConverter,
     IREE::LinalgExt::MaterializeEncodingValueFn materializeEncodingValueFn);
 
-// TODO(hanchung): Move the method to VMVX/EncodingInfo.h. This is required by
-// TileAndDistributeToWorkgroupPass and VMVXMaterializeEncodingPass. It can not
-// be in VMVX/EncodingInfo.h because there is a circular dependency. The Common/
-// should not depend on other target backends.
-FailureOr<IREE::LinalgExt::MaterializeEncodingValueInfo>
-chooseDynamicEncodingInfoVMVXMicrokernels(RankedTensorType tensorType,
-                                          OpBuilder &builder, Location loc);
-
 } // namespace iree_compiler
 } // namespace mlir
 #endif // IREE_COMPILER_SRC_IREE_COMPILER_CODEGEN_COMMON_ENCODINGINFO_H_
diff --git a/compiler/src/iree/compiler/Codegen/Common/MaterializeEncodingIntoPackUnPack.cpp b/compiler/src/iree/compiler/Codegen/Common/MaterializeEncodingIntoPackUnPack.cpp
index 278ce88..bcee05f 100644
--- a/compiler/src/iree/compiler/Codegen/Common/MaterializeEncodingIntoPackUnPack.cpp
+++ b/compiler/src/iree/compiler/Codegen/Common/MaterializeEncodingIntoPackUnPack.cpp
@@ -22,6 +22,7 @@
 #include "mlir/Dialect/Affine/IR/AffineOps.h"
 #include "mlir/Dialect/Arith/IR/Arith.h"
 #include "mlir/Dialect/MemRef/Transforms/Transforms.h"
+#include "mlir/IR/BuiltinTypes.h"
 #include "runtime/src/iree/builtins/ukernel/exported_bits.h"
 
 namespace mlir {
@@ -30,6 +31,23 @@
 using namespace IREE::LinalgExt;
 using IREE::HAL::ExecutableTargetAttr;
 
+static EncodingAttr getEncodingAttr(RankedTensorType type) {
+  return type.getEncoding().dyn_cast_or_null<EncodingAttr>();
+}
+
+static RankedTensorType getOriginalTypeWithEncoding(RankedTensorType type) {
+  auto encoding = getEncodingAttr(type);
+  if (!encoding) {
+    return type;
+  }
+  RankedTensorType originalType = type;
+  if (auto originalTypeAttr = encoding.getOriginalType()) {
+    originalType = originalTypeAttr.getValue().cast<RankedTensorType>();
+  }
+  return RankedTensorType::get(originalType.getShape(),
+                               originalType.getElementType(), encoding);
+}
+
 /// For `dispatchTensorType` that bind a `RankedTensorType` with encoding,
 /// returns the materialized shape of the `dispatchTensorType`. The
 /// dynamic dimensions of the `dispatchTensorType` are provided in
@@ -45,6 +63,9 @@
     return failure();
   }
 
+  RankedTensorType originalTensorType =
+      getOriginalTypeWithEncoding(boundTensorType);
+
   MaterializeEncodingFn materializeEncodingFn =
       typeConverter.getMaterializeEncodingFn();
   FailureOr<MaterializeEncodingInfo> encodingInfo =
@@ -54,9 +75,10 @@
   }
 
   SmallVector<OpFoldResult> targetShape =
-      getMixedValues(dispatchTensorType.getShape(), dynamicDims, builder);
-  auto innerTileSizes = getInnerTileSizesOfr(
-      builder, loc, boundTensorType, *encodingInfo, materializeEncodingValueFn);
+      getMixedValues(originalTensorType.getShape(), dynamicDims, builder);
+  auto innerTileSizes =
+      getInnerTileSizesOfr(builder, loc, originalTensorType, *encodingInfo,
+                           materializeEncodingValueFn);
   if (failed(innerTileSizes))
     return failure();
   SmallVector<OpFoldResult> convertedTargetShape =
@@ -216,6 +238,7 @@
     auto boundTensorType = targetType.getBoundType();
     auto *typeConverter =
         static_cast<MaterializeEncodingTypeConverter *>(getTypeConverter());
+
     if (typeConverter->convertType(boundTensorType) == boundTensorType) {
       return rewriter.notifyMatchFailure(storeOp, "bound type already valid");
     }
@@ -289,17 +312,38 @@
     // materializeEncodingValueFn which we obtain those tileSizes from.
     if (ShapedType::isDynamic(tileSize))
       continue;
-    auto generateNarrowTileSize = [&](int64_t n) {
-      if (size <= n && tileSize >= n)
-        tileSize = n;
-    };
-    generateNarrowTileSize(1);
-    generateNarrowTileSize(2);
-    generateNarrowTileSize(4);
+    // Adjust tile sizes for narrow cases: ensure that narrow sizes (those that
+    // are less than the normal tileSize) don't get padded to more than the
+    // next power of two, or tileSize, whichever is smaller.
+    //
+    // For example, if size==1, always adjust tileSize to be 1, so that
+    // matrix-times-vector problems remain that, instead of becoming more
+    // general matrix-times-matrix.
+    //
+    // Another example, if tileSize==6, then:
+    //
+    //   Original tensor size | adjusted tileSize
+    //   -------------------- | -----------------
+    //                      1 |                 1
+    //                      2 |                 2
+    //                      3 |                 4
+    //                      4 |                 4
+    //                      5 |                 6
+    //                   >= 6 |                 6
+    //
+    // Note: this implies that microkernels that implement a code path for
+    // a given `tileSize` value should also implement alternative code paths
+    // for all powers of two smaller than `tileSize`, as those could end up
+    // being selected here, and would fall back on slow generic code if no
+    // optimized code path is provided.
+    for (int po2 = 1; po2 < tileSize; po2 *= 2) {
+      if (size <= po2 && tileSize >= po2)
+        tileSize = po2;
+    }
   }
 }
 
-FailureOr<MaterializeEncodingValueInfo>
+static FailureOr<MaterializeEncodingValueInfo>
 chooseDynamicEncodingInfoVMVXMicrokernels(RankedTensorType tensorType,
                                           OpBuilder &builder, Location loc) {
   SmallVector<Type> resultTypes(tensorType.getRank(), builder.getIndexType());
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/BUILD.bazel b/compiler/src/iree/compiler/Codegen/LLVMCPU/BUILD.bazel
index fad7490..e4ff414 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/BUILD.bazel
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/BUILD.bazel
@@ -58,7 +58,6 @@
         "LLVMCPULinkExecutables.cpp",
         "LLVMCPULowerExecutableTarget.cpp",
         "LLVMCPULowerToUKernels.cpp",
-        "LLVMCPUMaterializeEncodingPass.cpp",
         "LLVMCPUMmt4dVectorLowering.cpp",
         "LLVMCPUPeel.cpp",
         "LLVMCPUSplitReduction.cpp",
@@ -88,6 +87,7 @@
         ":PassesIncGen",
         "//compiler/src/iree/compiler/Codegen/Common",
         "//compiler/src/iree/compiler/Codegen/Common:TransformDialectInterpreterPass",
+        "//compiler/src/iree/compiler/Codegen/Common/CPU:CommonCPUPasses",
         "//compiler/src/iree/compiler/Codegen/Dialect:IREECodegenDialect",
         "//compiler/src/iree/compiler/Codegen/Interfaces:PartitionableLoopsInterface",
         "//compiler/src/iree/compiler/Codegen/Interfaces:UKernelOpInterface",
@@ -104,6 +104,7 @@
         "//compiler/src/iree/compiler/Utils",
         "//llvm-external-projects/iree-dialects:IREELinalgExtDialect",
         "//llvm-external-projects/iree-dialects:IREELinalgExtPasses",
+        "//llvm-external-projects/iree-dialects:IREELinalgExtUtils",
         "//llvm-external-projects/iree-dialects:IREELinalgTransformDialect",
         "//llvm-external-projects/iree-dialects:IREELinalgTransformDialectPasses",
         "//runtime/src/iree/builtins/ukernel:exported_bits",
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/CMakeLists.txt b/compiler/src/iree/compiler/Codegen/LLVMCPU/CMakeLists.txt
index ba776c0..3939045 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/CMakeLists.txt
@@ -60,7 +60,6 @@
     "LLVMCPULinkExecutables.cpp"
     "LLVMCPULowerExecutableTarget.cpp"
     "LLVMCPULowerToUKernels.cpp"
-    "LLVMCPUMaterializeEncodingPass.cpp"
     "LLVMCPUMmt4dVectorLowering.cpp"
     "LLVMCPUPeel.cpp"
     "LLVMCPUSplitReduction.cpp"
@@ -81,6 +80,7 @@
     ::PassesIncGen
     IREELinalgExtDialect
     IREELinalgExtPasses
+    IREELinalgExtUtils
     IREELinalgTransformDialect
     IREELinalgTransformDialectPasses
     LLVMBinaryFormat
@@ -137,6 +137,7 @@
     MLIRVectorTransforms
     iree::builtins::ukernel::exported_bits
     iree::compiler::Codegen::Common
+    iree::compiler::Codegen::Common::CPU::CommonCPUPasses
     iree::compiler::Codegen::Common::TransformDialectInterpreterPass
     iree::compiler::Codegen::Dialect::IREECodegenDialect
     iree::compiler::Codegen::Interfaces::PartitionableLoopsInterface
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPULowerToUKernels.cpp b/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPULowerToUKernels.cpp
index bf24cae..9fa78c2 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPULowerToUKernels.cpp
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPULowerToUKernels.cpp
@@ -396,7 +396,7 @@
   auto encoding = tensorType.getEncoding()
                       .dyn_cast_or_null<IREE::LinalgExt::EncodingAttr>();
   if (!encoding) {
-    return rewriter.notifyMatchFailure(op, "no TensorEncoding attribute");
+    return rewriter.notifyMatchFailure(op, "no encoding attribute");
   }
   SmallVector<Type> resultTypes(tensorType.getRank(), rewriter.getIndexType());
   SmallVector<Value> inputValues;
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPUMaterializeEncodingPass.cpp b/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPUMaterializeEncodingPass.cpp
deleted file mode 100644
index 709583d..0000000
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/LLVMCPUMaterializeEncodingPass.cpp
+++ /dev/null
@@ -1,184 +0,0 @@
-// Copyright 2023 The IREE Authors
-//
-// Licensed under the Apache License v2.0 with LLVM Exceptions.
-// See https://llvm.org/LICENSE.txt for license information.
-// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
-
-#include "iree-dialects/Dialect/LinalgExt/IR/LinalgExtDialect.h"
-#include "iree-dialects/Dialect/LinalgExt/Passes/Passes.h"
-#include "iree/compiler/Codegen/Common/EncodingInfo.h"
-#include "iree/compiler/Codegen/Dialect/IREECodegenDialect.h"
-#include "iree/compiler/Codegen/LLVMCPU/PassDetail.h"
-#include "iree/compiler/Codegen/LLVMCPU/Utils.h"
-#include "iree/compiler/Dialect/Flow/IR/FlowOps.h"
-#include "iree/compiler/Dialect/HAL/IR/HALTypes.h"
-#include "mlir/Dialect/Affine/IR/AffineOps.h"
-#include "mlir/Dialect/Arith/IR/Arith.h"
-#include "mlir/Dialect/MemRef/Transforms/Transforms.h"
-#include "mlir/Dialect/Tensor/Transforms/Transforms.h"
-#include "mlir/Transforms/DialectConversion.h"
-#include "mlir/Transforms/GreedyPatternRewriteDriver.h"
-
-namespace mlir {
-namespace iree_compiler {
-
-using namespace IREE::LinalgExt;
-using IREE::HAL::ExecutableTargetAttr;
-
-namespace {
-
-static MatmulTileParams chooseMatmulTileParamsGeneric() { return {8, 4, 8}; }
-
-static MatmulTileParams
-chooseMatmulTileParamsAArch64(EncodingUser user, ExecutableTargetAttr target) {
-  switch (user) {
-  case EncodingUser::MATMUL_F32F32F32:
-  case EncodingUser::MATMUL_F16F16F32:
-  case EncodingUser::MATMUL_F16F16F16:
-  case EncodingUser::MATMUL_BF16BF16F32:
-  case EncodingUser::MATMUL_BF16BF16BF16:
-    // Note: 16-bit floating point types currently use the same tile size as
-    // f32. This makes sense when either (1) the accumulator is f32, or (2)
-    // the arithmetic will have to expand f16 to f32 in registers. We may
-    // reconsider when taking advantage of native f16/bf16 arithmetic when the
-    // accumulator itself is f16/bf16.
-    return {8, 1, 8};
-  case EncodingUser::MATMUL_I8I8I32:
-    if (hasFeature(target, "+i8mm")) {
-      // Aim to use SMMLA.
-      return {8, 8, 8};
-    }
-    if (hasFeature(target, "+dotprod")) {
-      // Aim to use SDOT.
-      return {8, 4, 8};
-    }
-    return {8, 1, 8};
-  default:
-    assert(false);
-    return {};
-  }
-}
-
-static MatmulTileParams
-chooseMatmulTileParamsX86_64(EncodingUser user, ExecutableTargetAttr target) {
-  switch (user) {
-  case EncodingUser::MATMUL_F32F32F32:
-  case EncodingUser::MATMUL_F16F16F32:
-  case EncodingUser::MATMUL_F16F16F16:
-  case EncodingUser::MATMUL_BF16BF16F32:
-  case EncodingUser::MATMUL_BF16BF16BF16:
-    // Note: 16-bit floating point types currently use the same tile size as
-    // f32. This makes sense when either (1) the accumulator is f32, or (2)
-    // the arithmetic will have to expand f16 to f32 in registers. We may
-    // reconsider when taking advantage of native f16/bf16 arithmetic when the
-    // accumulator itself is f16/bf16.
-    if (hasFeature(target, "+avx512f")) {
-      return {16, 1, 16};
-    }
-    if (hasFeature(target, "+avx")) {
-      // Note: for good performance, most +avx users will also want to add
-      // +fma, but that's a local instruction selection detail and the tile
-      // layout is unaffected, as there are enough registers even with the
-      // need for intermediate product registers when +fma is not used.
-      return {8, 1, 8};
-    }
-    // SSE fallback.
-    return {8, 1, 4};
-  case EncodingUser::MATMUL_I8I8I32:
-    if (hasFeature(target, "+avx512vnni")) {
-      // Aim to use VPDPWSSD. This is the same tile size as with VPMADDWD
-      // as the only difference is that VPDPWSSD accumulates. VPDPBUSD would
-      // call for {16, 4, 16} but we can't use it because of its unsigned LHS.
-      return {16, 2, 16};
-    }
-    if (hasFeature(target, "+avx512bw")) {
-      // Aim to use VPMADDWD (zmm).
-      return {16, 2, 16};
-    }
-    if (hasFeature(target, "+avx2")) {
-      // Aim to use VPMADDWD (ymm).
-      return {8, 2, 8};
-    }
-    // SSE fallback. Aim to use PMADDWD (xmm).
-    return {8, 2, 4};
-  default:
-    assert(false);
-    return {};
-  }
-}
-
-static MatmulTileParams chooseMatmulTileParams(EncodingUser user,
-                                               ExecutableTargetAttr target) {
-  if (isAArch64(target)) {
-    return chooseMatmulTileParamsAArch64(user, target);
-  }
-  if (isX86_64(target)) {
-    return chooseMatmulTileParamsX86_64(user, target);
-  }
-  return chooseMatmulTileParamsGeneric();
-}
-
-struct LLVMCPUMaterializeEncodingPass
-    : public LLVMCPUMaterializeEncodingBase<LLVMCPUMaterializeEncodingPass> {
-  void getDependentDialects(DialectRegistry &registry) const override {
-    registry
-        .insert<arith::ArithDialect, affine::AffineDialect,
-                IREE::Flow::FlowDialect, IREE::LinalgExt::IREELinalgExtDialect,
-                IREE::Codegen::IREECodegenDialect>();
-  }
-  void runOnOperation() override;
-};
-
-} // namespace
-
-void LLVMCPUMaterializeEncodingPass::runOnOperation() {
-  MLIRContext *context = &getContext();
-  auto operation = getOperation();
-  RewritePatternSet materializeEncodingPattern(context);
-  auto targetAttr = ExecutableTargetAttr::lookup(operation);
-  MaterializeEncodingTypeConverter typeConverter(
-      [targetAttr](
-          RankedTensorType tensorType) -> FailureOr<MaterializeEncodingInfo> {
-        auto encoding =
-            tensorType.getEncoding().dyn_cast_or_null<EncodingAttr>();
-        if (!encoding)
-          return failure();
-        auto user = encoding.getUser().getValue();
-        auto role = encoding.getRole().getValue();
-        MatmulTileParams tileParams = chooseMatmulTileParams(user, targetAttr);
-        auto encodingInfo = chooseEncodingInfoForMatmul(role, tileParams);
-        adjustTileSizesToNarrowStaticShape(encodingInfo, tensorType.getShape());
-        return encodingInfo;
-      });
-  MaterializeEncodingConversionTarget target(*context);
-  auto materializeEncodingValueFn = getMaterializeEncodingValueFn(targetAttr);
-  populateMaterializeEncodingIntoPackUnPackPatterns(materializeEncodingPattern,
-                                                    target, typeConverter,
-                                                    materializeEncodingValueFn);
-
-  if (failed(applyPartialConversion(operation, target,
-                                    std::move(materializeEncodingPattern)))) {
-    operation.emitOpError("materialization failed");
-    return signalPassFailure();
-  }
-
-  // Add patterns to fold pack/unpack ops with pad/extract_slice ops and resolve
-  // dims ops.
-  {
-    RewritePatternSet patterns(context);
-    tensor::populateFoldIntoPackAndUnpackPatterns(patterns);
-    memref::populateResolveRankedShapedTypeResultDimsPatterns(patterns);
-    if (failed(applyPatternsAndFoldGreedily(operation, std::move(patterns)))) {
-      operation.emitOpError("folding patterns failed");
-      return signalPassFailure();
-    }
-  }
-}
-
-std::unique_ptr<OperationPass<func::FuncOp>>
-createLLVMCPUMaterializeEncodingPass() {
-  return std::make_unique<LLVMCPUMaterializeEncodingPass>();
-}
-
-} // namespace iree_compiler
-} // namespace mlir
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.cpp b/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.cpp
index c0bc01c..52830f5 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.cpp
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.cpp
@@ -5,8 +5,8 @@
 // SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
 
 #include "iree-dialects/Dialect/LinalgExt/Passes/Passes.h"
-
 #include "iree-dialects/Dialect/LinalgTransform/Passes.h"
+#include "iree/compiler/Codegen/Common/CPU/Passes.h"
 #include "iree/compiler/Codegen/Common/Passes.h"
 #include "iree/compiler/Codegen/Interfaces/PartitionableLoopsInterface.h"
 #include "iree/compiler/Codegen/LLVMCPU/KernelDispatch.h"
@@ -15,7 +15,6 @@
 #include "iree/compiler/Codegen/Transforms/Transforms.h"
 #include "iree/compiler/Codegen/Utils/Utils.h"
 #include "iree/compiler/Codegen/VMVX/Passes.h"
-#include "iree/compiler/Dialect/Util/Transforms/Passes.h"
 #include "llvm/ADT/TypeSwitch.h"
 #include "llvm/Support/CommandLine.h"
 #include "mlir/Conversion/ComplexToStandard/ComplexToStandard.h"
@@ -746,7 +745,7 @@
   passManager.addPass(createExpandF16OpToF32Pass());
 
   passManager.nest<ModuleOp>().addNestedPass<func::FuncOp>(
-      createLLVMCPUMaterializeEncodingPass());
+      createCPUMaterializeEncodingPass());
   // TODO: Remove the following pass the plumb support for #hal.descriptor_type
   // memory space through the stack.
   passManager.nest<ModuleOp>().addNestedPass<func::FuncOp>(
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.h b/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.h
index 4f30ac1..d99a262 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.h
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.h
@@ -47,11 +47,6 @@
 /// operation.
 std::unique_ptr<OperationPass<>> createLLVMCPULowerToUKernelsPass();
 
-/// Materialize the encoding of operations. The layout to use for the encoded
-/// operations are LLVMCPU specific.
-std::unique_ptr<OperationPass<func::FuncOp>>
-createLLVMCPUMaterializeEncodingPass();
-
 std::unique_ptr<OperationPass<func::FuncOp>>
 createLLVMCPUMmt4dVectorLoweringPass();
 
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.td b/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.td
index 6dd2edb..f73ef62 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.td
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/Passes.td
@@ -81,12 +81,6 @@
       "mlir::iree_compiler::createLLVMCPULowerToUKernelsPass()";
 }
 
-def LLVMCPUMaterializeEncoding :
-    Pass<"iree-llvmcpu-materialize-encoding", "func::FuncOp"> {
-  let summary = "Materialize the encoding for tensor as specified by the backend";
-  let constructor = "mlir::iree_compiler::createLLVMCPUMaterializeEncodingPass()";
-}
-
 def LLVMCPUMmt4dVectorLowering
     : Pass<"iree-llvmcpu-mmt4d-vector-lowering", "func::FuncOp"> {
   let summary = "Apply vector lowering logic to vector ops";
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.cpp b/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.cpp
index 58ded71..eaaa3c3 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.cpp
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.cpp
@@ -14,63 +14,12 @@
 namespace mlir {
 namespace iree_compiler {
 
-std::optional<StringRef>
-getCpuFeatures(IREE::HAL::ExecutableTargetAttr targetAttr) {
-  auto cpuFeatures = getConfigStringAttr(targetAttr, "cpu_features");
-  if (!cpuFeatures)
-    return std::nullopt;
-  return cpuFeatures->getValue();
-}
-
-bool isX86(IREE::HAL::ExecutableTargetAttr targetAttr) {
-  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
-  return triple && triple.value().isX86();
-}
-
-bool isX86_64(IREE::HAL::ExecutableTargetAttr targetAttr) {
-  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
-  return triple && triple.value().getArch() == llvm::Triple::x86_64;
-}
-
-bool isAArch64(IREE::HAL::ExecutableTargetAttr targetAttr) {
-  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
-  return triple && triple.value().isAArch64();
-}
-
-bool isRISCV(IREE::HAL::ExecutableTargetAttr targetAttr) {
-  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
-  return triple && triple.value().isRISCV();
-}
-
 bool preferIntrinsicsOverAsm(IREE::HAL::ExecutableTargetAttr targetAttr) {
   auto intrinsicsAttr =
       getConfigBoolAttr(targetAttr, "prefer_intrinsics_over_asm");
   return intrinsicsAttr && intrinsicsAttr->getValue();
 }
 
-// TODO(dcaballe): If we have to check for a significantly large number of
-// features in the future, we may want to consider a persistent state to carry
-// over processed HAL information or keeping the TTI instance alive and query
-// subtarget features data structure.
-bool hasFeature(IREE::HAL::ExecutableTargetAttr targetAttr, StringRef feature) {
-  std::optional<StringRef> features = getCpuFeatures(targetAttr);
-  if (!features) {
-    return false;
-  }
-
-  // Find feature string in list of features, making sure that we don't match a
-  // sub-string.
-  std::stringstream sstream(features->str());
-  std::string str;
-  while (std::getline(sstream, str, ',')) {
-    if (str == feature) {
-      return true;
-    }
-  }
-
-  return false;
-}
-
 bool hasAVX2Feature(IREE::HAL::ExecutableTargetAttr targetAttr) {
   return hasFeature(targetAttr, "+avx2");
 }
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.h b/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.h
index 06aaaee..a8f771a 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.h
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/Utils.h
@@ -7,25 +7,14 @@
 #ifndef IREE_COMPILER_CODEGEN_LLVMCPU_UTILS_H_
 #define IREE_COMPILER_CODEGEN_LLVMCPU_UTILS_H_
 
+#include "iree/compiler/Codegen/Utils/Utils.h"
 #include "iree/compiler/Dialect/HAL/IR/HALOps.h"
 
 namespace mlir {
 namespace iree_compiler {
 
-/// Returns the CPU target features associated with the `targetAttr`, if set.
-std::optional<StringRef>
-getCpuFeatures(IREE::HAL::ExecutableTargetAttr targetAttr);
-
-/// Methods to get target information.
-bool isX86(IREE::HAL::ExecutableTargetAttr targetAttr);
-bool isX86_64(IREE::HAL::ExecutableTargetAttr targetAttr);
-bool isAArch64(IREE::HAL::ExecutableTargetAttr targetAttr);
-bool isRISCV(IREE::HAL::ExecutableTargetAttr targetAttr);
 bool preferIntrinsicsOverAsm(IREE::HAL::ExecutableTargetAttr targetAttr);
 
-/// Returns true if `targetAttr` has `feature` in its CPU features.
-bool hasFeature(IREE::HAL::ExecutableTargetAttr targetAttr, StringRef feature);
-
 /// Returns true if the 'targetAttr' contains '+avx2' in its cpu features.
 bool hasAVX2Feature(IREE::HAL::ExecutableTargetAttr targetAttr);
 
diff --git a/compiler/src/iree/compiler/Codegen/LLVMCPU/test/materialize_encoding.mlir b/compiler/src/iree/compiler/Codegen/LLVMCPU/test/materialize_encoding.mlir
index da45574..a23012f 100644
--- a/compiler/src/iree/compiler/Codegen/LLVMCPU/test/materialize_encoding.mlir
+++ b/compiler/src/iree/compiler/Codegen/LLVMCPU/test/materialize_encoding.mlir
@@ -1,6 +1,47 @@
-// RUN: iree-opt --iree-llvmcpu-materialize-encoding --canonicalize --cse --split-input-file %s | FileCheck %s
+// RUN: iree-opt --iree-cpu-materialize-encoding --canonicalize --cse --split-input-file %s | FileCheck %s
 
-func.func @set_encoding_op() {
+func.func @set_encoding_7x7x7_matmul_LHS() attributes {
+   hal.executable.target = #hal.executable.target<"xyz", "xyz", {target_triple="x86_64-xyz-xyz", cpu_features="+avx,+avx2,+fma"}>
+} {
+  %cst = arith.constant 0.000000e+00 : f32
+  %c0 = arith.constant 0 : index
+  %0 = hal.interface.constant.load[0] : i32
+  %1 = hal.interface.constant.load[1] : i32
+  %2 = hal.interface.constant.load[2] : i32
+  %3 = hal.interface.constant.load[3] : i32
+  %4 = arith.index_castui %0 : i32 to index
+  %5 = arith.index_castui %1 : i32 to index
+  %6 = arith.index_castui %2 : i32 to index
+  %7 = arith.index_castui %3 : i32 to index
+  %8 = hal.interface.binding.subspan set(0) binding(0) type(storage_buffer) alignment(64) offset(%c0) flags(ReadOnly) : !flow.dispatch.tensor<readonly:tensor<7x7xf32>>
+  %9 = flow.dispatch.workload.ordinal %6, 2 : index
+  %10 = flow.dispatch.workload.ordinal %7, 3 : index
+  %11 = hal.interface.binding.subspan set(0) binding(1) type(storage_buffer) alignment(64) offset(%c0) : !flow.dispatch.tensor<writeonly:tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS, original_type = tensor<7x7xf32>>>>{%9, %10}
+  %12 = flow.dispatch.workload.ordinal %4, 0 : index
+  %13 = flow.dispatch.workload.ordinal %5, 1 : index
+  %14 = flow.dispatch.tensor.load %8, offsets = [0, 0], sizes = [7, 7], strides = [1, 1] : !flow.dispatch.tensor<readonly:tensor<7x7xf32>> -> tensor<7x7xf32>
+  %15 = affine.apply affine_map<()[s0] -> ((7 ceildiv s0) * s0 - 7)>()[%12]
+  %16 = affine.apply affine_map<()[s0] -> ((7 ceildiv s0) * s0 - 7)>()[%13]
+  %padded = tensor.pad %14 low[0, 0] high[%15, %16] {
+  ^bb0(%arg0: index, %arg1: index):
+    tensor.yield %cst : f32
+  } : tensor<7x7xf32> to tensor<?x?xf32>
+  %17 = iree_linalg_ext.set_encoding %padded : tensor<?x?xf32> -> tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS, original_type = tensor<7x7xf32>>>
+  flow.dispatch.tensor.store %17, %11, offsets = [0, 0], sizes = [%9, %10], strides = [1, 1] : tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS, original_type = tensor<7x7xf32>>> -> !flow.dispatch.tensor<writeonly:tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS, original_type = tensor<7x7xf32>>>>{%9, %10}
+  return
+}
+// CHECK:    func @set_encoding_7x7x7_matmul_LHS(
+// CHECK-DAG:  %[[CST:.+]] = arith.constant 0.0
+// CHECK:      %[[INPUT_BINDING:.+]] = hal.interface.binding.subspan {{.*}} !flow.dispatch.tensor<readonly:tensor<7x7xf32>>
+// CHECK:      %[[OUTPUT_BINDING:.+]] = hal.interface.binding.subspan {{.*}} !flow.dispatch.tensor<writeonly:tensor<1x7x8x1xf32>>
+// CHECK:      %[[INPUT:.+]] = flow.dispatch.tensor.load %[[INPUT_BINDING]], offsets = [0, 0], sizes = [7, 7], strides = [1, 1] : !flow.dispatch.tensor<readonly:tensor<7x7xf32>> -> tensor<7x7xf32>
+// CHECK:      %[[EMPTY:.+]] = tensor.empty() : tensor<1x7x8x1xf32>
+// CHECK:      %[[PACK:.+]] = tensor.pack %[[INPUT]] padding_value(%[[CST]] : f32) inner_dims_pos = [0, 1] inner_tiles = [8, 1] into %3 : tensor<7x7xf32> -> tensor<1x7x8x1xf32>
+// CHECK:      flow.dispatch.tensor.store %[[PACK]], %[[OUTPUT_BINDING]], offsets = [0, 0, 0, 0], sizes = [1, 7, 8, 1], strides = [1, 1, 1, 1] : tensor<1x7x8x1xf32> -> !flow.dispatch.tensor<writeonly:tensor<1x7x8x1xf32>>
+
+// -----
+
+func.func @set_encoding_dynamic() {
   %c0 = arith.constant 0 : index
   %cst = arith.constant 0.000000e+00 : f32
   %d0 = hal.interface.constant.load [0] : index
@@ -27,7 +68,7 @@
 }
 //   CHECK-DAG: #[[MAP0:.+]] = affine_map<()[s0] -> (s0 ceildiv 8)>
 //   CHECK-DAG: #[[MAP1:.+]] = affine_map<()[s0] -> (s0 ceildiv 4)>
-//       CHECK: func @set_encoding_op()
+//       CHECK: func @set_encoding_dynamic()
 //   CHECK-DAG:   %[[C0:.+]] = arith.constant 0 : index
 //   CHECK-DAG:   %[[CST:.+]] = arith.constant 0.0
 //   CHECK-DAG:   %[[D0:.+]] = hal.interface.constant.load[0]
@@ -40,7 +81,7 @@
 //   CHECK-DAG:   %[[OUTPUT_BINDING:.+]] = hal.interface.binding.subspan set(0) binding(1)
 //  CHECK-SAME:       !flow.dispatch.tensor<writeonly:tensor<?x?x8x4xf32>>{%[[TILED_OUTD0]], %[[TILED_OUTD1]]}
 //       CHECK:   %[[INPUT:.+]] = flow.dispatch.tensor.load %[[INPUT_BINDING]]
-//       CHECK:   %[[EMPTY:.+]] = tensor.empty(%[[TILED_OUTD0]], %[[TILED_OUTD1]])
+//       CHECK:   %[[EMPTY:.+]] = tensor.empty
 //       CHECK:   %[[PACK:.+]] = tensor.pack
 //  CHECK-SAME:       %[[INPUT]] padding_value(%[[CST]] : f32)
 //  CHECK-SAME:       inner_dims_pos = [0, 1] inner_tiles = [8, 4] into %[[EMPTY]]
@@ -49,7 +90,7 @@
 
 // -----
 
-func.func @unset_encoding_op() {
+func.func @unset_encoding_dynamic() {
   %c0 = arith.constant 0 : index
   %cst = arith.constant 0.000000e+00 : f32
   %d0 = hal.interface.constant.load [0] : index
@@ -72,7 +113,7 @@
 }
 //   CHECK-DAG: #[[MAP0:.+]] = affine_map<()[s0] -> (s0 ceildiv 8)>
 //   CHECK-DAG: #[[MAP1:.+]] = affine_map<()[s0] -> (s0 ceildiv 4)>
-//       CHECK: func @unset_encoding_op()
+//       CHECK: func @unset_encoding_dynamic()
 //   CHECK-DAG:   %[[C0:.+]] = arith.constant 0 : index
 //   CHECK-DAG:   %[[D0:.+]] = hal.interface.constant.load[0]
 //   CHECK-DAG:   %[[D1:.+]] = hal.interface.constant.load[1]
@@ -212,6 +253,57 @@
 
 // -----
 
+func.func @matvec_lowering_f32f32f32_aarch64() attributes {
+  hal.executable.target = #hal.executable.target<"xyz", "xyz", {target_triple="aarch64-xyz-xyz"}>
+} {
+  %c0 = arith.constant 0 : index
+  %0 = hal.interface.binding.subspan set(0) binding(0) type(storage_buffer) alignment(64) offset(%c0)
+      : !flow.dispatch.tensor<readonly:tensor<16x16xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>>
+  %1 = hal.interface.binding.subspan set(0) binding(1) type(storage_buffer) alignment(64) offset(%c0)
+      : !flow.dispatch.tensor<readonly:tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>>
+  %2 = hal.interface.binding.subspan set(0) binding(2) type(storage_buffer) alignment(64) offset(%c0)
+      : !flow.dispatch.tensor<readwrite:tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>>
+  %3 = flow.dispatch.tensor.load %0, offsets = [0, 0], sizes = [16, 16], strides = [1, 1]
+      : !flow.dispatch.tensor<readonly:tensor<16x16xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>>
+      -> tensor<16x16xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>
+  %4 = flow.dispatch.tensor.load %1, offsets = [0, 0], sizes = [16, 1], strides = [1, 1]
+      : !flow.dispatch.tensor<readonly:tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>>
+      -> tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>
+  %5 = flow.dispatch.tensor.load %2, offsets = [0, 0], sizes = [16, 1], strides = [1, 1]
+      : !flow.dispatch.tensor<readwrite:tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>>
+      -> tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
+  %6 = linalg.matmul
+      ins(%3, %4 : tensor<16x16xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>,
+                   tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>)
+      outs(%5 : tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>)
+      -> tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
+  flow.dispatch.tensor.store %6, %2, offsets = [0, 0], sizes = [16, 1], strides = [1, 1]
+      : tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
+      -> !flow.dispatch.tensor<readwrite:tensor<16x1xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>>
+  return
+}
+//      CHECK: func @matvec_lowering_f32f32f32_aarch64()
+//  CHECK-DAG:   %[[C0:.+]] = arith.constant 0 : index
+//      CHECK:   %[[LHS_BINDING:.+]] = hal.interface.binding.subspan set(0) binding(0)
+// CHECK-SAME:       !flow.dispatch.tensor<readonly:tensor<2x16x8x1xf32>>
+//      CHECK:   %[[RHS_BINDING:.+]] = hal.interface.binding.subspan set(0) binding(1)
+// CHECK-SAME:       !flow.dispatch.tensor<readonly:tensor<1x16x1x1xf32>>
+//      CHECK:   %[[OUTS_BINDING:.+]] = hal.interface.binding.subspan set(0) binding(2)
+// CHECK-SAME:       !flow.dispatch.tensor<readwrite:tensor<2x1x8x1xf32>>
+//      CHECK:   %[[LHS:.+]] = flow.dispatch.tensor.load %[[LHS_BINDING]]
+// CHECK-SAME:       offsets = [0, 0, 0, 0], sizes = [2, 16, 8, 1], strides = [1, 1, 1, 1]
+//      CHECK:   %[[RHS:.+]] = flow.dispatch.tensor.load %[[RHS_BINDING]]
+// CHECK-SAME:       offsets = [0, 0, 0, 0], sizes = [1, 16, 1, 1], strides = [1, 1, 1, 1]
+//      CHECK:   %[[OUTS:.+]] = flow.dispatch.tensor.load %[[OUTS_BINDING]]
+// CHECK-SAME:       offsets = [0, 0, 0, 0], sizes = [2, 1, 8, 1], strides = [1, 1, 1, 1]
+//      CHECK:   %[[MMT4D:.+]] = linalg.mmt4d
+// CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
+// CHECK-SAME:       outs(%[[OUTS]] :
+//      CHECK:   flow.dispatch.tensor.store %[[MMT4D]], %[[OUTS_BINDING]]
+// CHECK-SAME:       offsets = [0, 0, 0, 0], sizes = [2, 1, 8, 1], strides = [1, 1, 1, 1]
+
+// -----
+
 func.func @matmul_lowering_f16f16f16_aarch64() attributes {
   hal.executable.target = #hal.executable.target<"xyz", "xyz", {target_triple="aarch64-xyz-xyz"}>
 } {
diff --git a/compiler/src/iree/compiler/Codegen/Passes.cpp b/compiler/src/iree/compiler/Codegen/Passes.cpp
index 293ea73..c185a75 100644
--- a/compiler/src/iree/compiler/Codegen/Passes.cpp
+++ b/compiler/src/iree/compiler/Codegen/Passes.cpp
@@ -9,6 +9,7 @@
 //===---------------------------------------------------------------------===//
 // Include pass headers per target device
 //===---------------------------------------------------------------------===//
+#include "iree/compiler/Codegen/Common/CPU/Passes.h"
 #include "iree/compiler/Codegen/Common/GPU/Passes.h"
 #include "iree/compiler/Codegen/Common/Passes.h"
 #include "iree/compiler/Codegen/LLVMCPU/Passes.h"
@@ -23,6 +24,7 @@
 void registerCodegenPasses() {
   // Generated.
   registerCodegenCommonPasses();
+  registerCodegenCommonCPUPasses();
   registerCodegenCommonGPUPasses();
   registerCodegenLLVMCPUPasses();
   registerCodegenLLVMGPUPasses();
diff --git a/compiler/src/iree/compiler/Codegen/Utils/Utils.cpp b/compiler/src/iree/compiler/Codegen/Utils/Utils.cpp
index be3b2b2..370cbd7 100644
--- a/compiler/src/iree/compiler/Codegen/Utils/Utils.cpp
+++ b/compiler/src/iree/compiler/Codegen/Utils/Utils.cpp
@@ -152,6 +152,57 @@
   return enableMicrokernels && enableMicrokernels->getValue();
 }
 
+std::optional<StringRef>
+getCpuFeatures(IREE::HAL::ExecutableTargetAttr targetAttr) {
+  auto cpuFeatures = getConfigStringAttr(targetAttr, "cpu_features");
+  if (!cpuFeatures)
+    return std::nullopt;
+  return cpuFeatures->getValue();
+}
+
+// TODO(dcaballe): If we have to check for a significantly large number of
+// features in the future, we may want to consider a persistent state to carry
+// over processed HAL information or keeping the TTI instance alive and query
+// subtarget features data structure.
+bool hasFeature(IREE::HAL::ExecutableTargetAttr targetAttr, StringRef feature) {
+  std::optional<StringRef> features = getCpuFeatures(targetAttr);
+  if (!features) {
+    return false;
+  }
+
+  // Find feature string in list of features, making sure that we don't match a
+  // sub-string.
+  std::stringstream sstream(features->str());
+  std::string str;
+  while (std::getline(sstream, str, ',')) {
+    if (str == feature) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool isX86(IREE::HAL::ExecutableTargetAttr targetAttr) {
+  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
+  return triple && triple.value().isX86();
+}
+
+bool isX86_64(IREE::HAL::ExecutableTargetAttr targetAttr) {
+  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
+  return triple && triple.value().getArch() == llvm::Triple::x86_64;
+}
+
+bool isAArch64(IREE::HAL::ExecutableTargetAttr targetAttr) {
+  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
+  return triple && triple.value().isAArch64();
+}
+
+bool isRISCV(IREE::HAL::ExecutableTargetAttr targetAttr) {
+  std::optional<llvm::Triple> triple = getTargetTriple(targetAttr);
+  return triple && triple.value().isRISCV();
+}
+
 bool isReadOnly(Value v) {
   Operation *definingOp = v.getDefiningOp();
   if (!definingOp)
diff --git a/compiler/src/iree/compiler/Codegen/Utils/Utils.h b/compiler/src/iree/compiler/Codegen/Utils/Utils.h
index 23000c8..ab4961f 100644
--- a/compiler/src/iree/compiler/Codegen/Utils/Utils.h
+++ b/compiler/src/iree/compiler/Codegen/Utils/Utils.h
@@ -74,6 +74,19 @@
 bool isVMVXBackend(IREE::HAL::ExecutableTargetAttr targetAttr);
 bool hasMicrokernels(IREE::HAL::ExecutableTargetAttr targetAttr);
 
+/// Returns the CPU target features associated with the `targetAttr`, if set.
+std::optional<StringRef>
+getCpuFeatures(IREE::HAL::ExecutableTargetAttr targetAttr);
+
+/// Returns true if `targetAttr` has `feature` in its CPU features.
+bool hasFeature(IREE::HAL::ExecutableTargetAttr targetAttr, StringRef feature);
+
+// Architecture identification.
+bool isX86(IREE::HAL::ExecutableTargetAttr targetAttr);
+bool isX86_64(IREE::HAL::ExecutableTargetAttr targetAttr);
+bool isAArch64(IREE::HAL::ExecutableTargetAttr targetAttr);
+bool isRISCV(IREE::HAL::ExecutableTargetAttr targetAttr);
+
 /// Checks if a tensor value is generated from a read-only object, like
 /// and interface binding with read-only attribute or from an `arith.constant`
 /// operation.
diff --git a/compiler/src/iree/compiler/Codegen/VMVX/BUILD.bazel b/compiler/src/iree/compiler/Codegen/VMVX/BUILD.bazel
index 763c922..3cf3f89 100644
--- a/compiler/src/iree/compiler/Codegen/VMVX/BUILD.bazel
+++ b/compiler/src/iree/compiler/Codegen/VMVX/BUILD.bazel
@@ -51,10 +51,8 @@
         "Passes.cpp",
         "VMVXAssignConstantOrdinals.cpp",
         "VMVXLinkExecutables.cpp",
-        "VMVXMaterializeEncodingPass.cpp",
     ],
     hdrs = [
-        "EncodingInfo.h",
         "Passes.h",
     ],
     deps = [
diff --git a/compiler/src/iree/compiler/Codegen/VMVX/CMakeLists.txt b/compiler/src/iree/compiler/Codegen/VMVX/CMakeLists.txt
index 3d1b599..f19c024 100644
--- a/compiler/src/iree/compiler/Codegen/VMVX/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Codegen/VMVX/CMakeLists.txt
@@ -42,14 +42,12 @@
   NAME
     VMVX
   HDRS
-    "EncodingInfo.h"
     "Passes.h"
   SRCS
     "LowerLinalgMicrokernels.cpp"
     "Passes.cpp"
     "VMVXAssignConstantOrdinals.cpp"
     "VMVXLinkExecutables.cpp"
-    "VMVXMaterializeEncodingPass.cpp"
   DEPS
     ::PassHeaders
     ::PassesIncGen
diff --git a/compiler/src/iree/compiler/Codegen/VMVX/EncodingInfo.h b/compiler/src/iree/compiler/Codegen/VMVX/EncodingInfo.h
deleted file mode 100644
index 1a52720..0000000
--- a/compiler/src/iree/compiler/Codegen/VMVX/EncodingInfo.h
+++ /dev/null
@@ -1,19 +0,0 @@
-// Copyright 2023 The IREE Authors
-//
-// Licensed under the Apache License v2.0 with LLVM Exceptions.
-// See https://llvm.org/LICENSE.txt for license information.
-// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
-
-#ifndef IREE_COMPILER_CODEGEN_VMVX_ENCODINGINFO_H_
-#define IREE_COMPILER_CODEGEN_VMVX_ENCODINGINFO_H_
-
-#include "iree/compiler/Codegen/Common/EncodingInfo.h"
-
-namespace mlir {
-namespace iree_compiler {
-// A placeholder for chooseDynamicEncodingInfoVMVXMicrokernels. See
-// Common/EncodingInfo.h for more details.
-} // namespace iree_compiler
-} // namespace mlir
-
-#endif // IREE_COMPILER_CODEGEN_VMVX_ENCODINGINFO_H_
diff --git a/compiler/src/iree/compiler/Codegen/VMVX/Passes.td b/compiler/src/iree/compiler/Codegen/VMVX/Passes.td
index 92ea2fd..e8c240c 100644
--- a/compiler/src/iree/compiler/Codegen/VMVX/Passes.td
+++ b/compiler/src/iree/compiler/Codegen/VMVX/Passes.td
@@ -37,10 +37,4 @@
   ];
 }
 
-def VMVXMaterializeEncoding :
-    Pass<"iree-vmvx-materialize-encoding", "func::FuncOp"> {
-  let summary = "Materialize the encoding for tensor as specified by the backend";
-  let constructor = "mlir::iree_compiler::createVMVXMaterializeEncodingPass()";
-}
-
 #endif // IREE_CODEGEN_VMVX_PASSES
diff --git a/compiler/src/iree/compiler/Codegen/VMVX/VMVXMaterializeEncodingPass.cpp b/compiler/src/iree/compiler/Codegen/VMVX/VMVXMaterializeEncodingPass.cpp
deleted file mode 100644
index 3b83b5b..0000000
--- a/compiler/src/iree/compiler/Codegen/VMVX/VMVXMaterializeEncodingPass.cpp
+++ /dev/null
@@ -1,117 +0,0 @@
-// Copyright 2023 The IREE Authors
-//
-// Licensed under the Apache License v2.0 with LLVM Exceptions.
-// See https://llvm.org/LICENSE.txt for license information.
-// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
-
-#include "iree-dialects/Dialect/LinalgExt/IR/LinalgExtDialect.h"
-#include "iree-dialects/Dialect/LinalgExt/Passes/Passes.h"
-#include "iree/compiler/Codegen/Common/EncodingInfo.h"
-#include "iree/compiler/Codegen/Dialect/IREECodegenDialect.h"
-#include "iree/compiler/Codegen/Utils/Utils.h"
-#include "iree/compiler/Codegen/VMVX/EncodingInfo.h"
-#include "iree/compiler/Codegen/VMVX/PassDetail.h"
-#include "iree/compiler/Codegen/VMVX/Passes.h"
-#include "iree/compiler/Dialect/Flow/IR/FlowOps.h"
-#include "iree/compiler/Dialect/HAL/IR/HALTypes.h"
-#include "iree/compiler/Dialect/VMVX/IR/VMVXDialect.h"
-#include "iree/compiler/Dialect/VMVX/IR/VMVXOps.h"
-#include "mlir/Dialect/Affine/IR/AffineOps.h"
-#include "mlir/Dialect/Arith/IR/Arith.h"
-#include "mlir/Dialect/MemRef/Transforms/Transforms.h"
-#include "mlir/Dialect/Tensor/Transforms/Transforms.h"
-#include "mlir/Transforms/DialectConversion.h"
-#include "mlir/Transforms/GreedyPatternRewriteDriver.h"
-
-namespace mlir {
-namespace iree_compiler {
-
-using namespace IREE::LinalgExt;
-using IREE::HAL::ExecutableTargetAttr;
-
-namespace {
-
-static MatmulTileParams chooseMatmulTileParamsGeneric() { return {8, 4, 8}; }
-
-static MatmulTileParams chooseMicrokernelMatmulTileParams() {
-  return {ShapedType::kDynamic, ShapedType::kDynamic, ShapedType::kDynamic};
-}
-
-static MatmulTileParams chooseMatmulTileParams(ExecutableTargetAttr target) {
-  if (hasMicrokernels(target)) {
-    return chooseMicrokernelMatmulTileParams();
-  }
-  return chooseMatmulTileParamsGeneric();
-}
-
-static MaterializeEncodingValueFn
-getMaterializeEncodingValueFn(IREE::HAL::ExecutableTargetAttr targetAttr) {
-  if (hasMicrokernels(targetAttr)) {
-    return chooseDynamicEncodingInfoVMVXMicrokernels;
-  }
-  return {};
-}
-
-struct VMVXMaterializeEncodingPass
-    : public VMVXMaterializeEncodingBase<VMVXMaterializeEncodingPass> {
-  void getDependentDialects(DialectRegistry &registry) const override {
-    registry.insert<
-        arith::ArithDialect, affine::AffineDialect, tensor::TensorDialect,
-        IREE::Flow::FlowDialect, IREE::LinalgExt::IREELinalgExtDialect,
-        IREE::VMVX::VMVXDialect, IREE::Codegen::IREECodegenDialect>();
-  }
-  void runOnOperation() override;
-};
-
-} // namespace
-
-void VMVXMaterializeEncodingPass::runOnOperation() {
-  MLIRContext *context = &getContext();
-  auto operation = getOperation();
-  RewritePatternSet materializeEncodingPattern(context);
-  auto targetAttr = ExecutableTargetAttr::lookup(operation);
-  MaterializeEncodingTypeConverter typeConverter(
-      [targetAttr](
-          RankedTensorType tensorType) -> FailureOr<MaterializeEncodingInfo> {
-        auto encoding = tensorType.getEncoding()
-                            .dyn_cast_or_null<IREE::LinalgExt::EncodingAttr>();
-        if (!encoding)
-          return failure();
-        auto role = encoding.getRole().getValue();
-        MatmulTileParams tileParams = chooseMatmulTileParams(targetAttr);
-        auto encodingInfo = chooseEncodingInfoForMatmul(role, tileParams);
-        adjustTileSizesToNarrowStaticShape(encodingInfo, tensorType.getShape());
-        return encodingInfo;
-      });
-  MaterializeEncodingConversionTarget target(*context);
-  auto materializeEncodingValueFn = getMaterializeEncodingValueFn(targetAttr);
-  populateMaterializeEncodingIntoPackUnPackPatterns(materializeEncodingPattern,
-                                                    target, typeConverter,
-                                                    materializeEncodingValueFn);
-
-  if (failed(applyPartialConversion(operation, target,
-                                    std::move(materializeEncodingPattern)))) {
-    operation.emitOpError("materialization failed");
-    return signalPassFailure();
-  }
-
-  // Add patterns to fold pack/unpack ops with pad/extract_slice ops and resolve
-  // dims ops.
-  {
-    RewritePatternSet patterns(context);
-    tensor::populateFoldIntoPackAndUnpackPatterns(patterns);
-    memref::populateResolveRankedShapedTypeResultDimsPatterns(patterns);
-    if (failed(applyPatternsAndFoldGreedily(operation, std::move(patterns)))) {
-      operation.emitOpError("folding patterns failed");
-      return signalPassFailure();
-    }
-  }
-}
-
-std::unique_ptr<OperationPass<func::FuncOp>>
-createVMVXMaterializeEncodingPass() {
-  return std::make_unique<VMVXMaterializeEncodingPass>();
-}
-
-} // namespace iree_compiler
-} // namespace mlir
diff --git a/compiler/src/iree/compiler/Codegen/VMVX/test/materialize_encoding.mlir b/compiler/src/iree/compiler/Codegen/VMVX/test/materialize_encoding.mlir
index a3c2137..e0fc241 100644
--- a/compiler/src/iree/compiler/Codegen/VMVX/test/materialize_encoding.mlir
+++ b/compiler/src/iree/compiler/Codegen/VMVX/test/materialize_encoding.mlir
@@ -1,4 +1,4 @@
-// RUN: iree-opt --iree-vmvx-materialize-encoding --canonicalize --cse --split-input-file %s | FileCheck %s
+// RUN: iree-opt --iree-cpu-materialize-encoding --canonicalize --cse --split-input-file %s | FileCheck %s
 
 func.func @matmul_lowering_i8i8i32_vmvx_ukernel() attributes {
   hal.executable.target = #hal.executable.target<"vmvx", "vmvx-bytecode-fb", {ukernels = true}>
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/FormDispatchRegions.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/FormDispatchRegions.cpp
index 90b671a..609c170 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/FormDispatchRegions.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/FormDispatchRegions.cpp
@@ -221,11 +221,28 @@
   return llvm::cast<RankedTensorType>(source.getType());
 }
 
-/// Returns true if the operation is an `unpack` op or an `unset_encoding` op
-/// that has unpack semantics
-// TODO(ravishankarm): This seems like a use case for interface.
-static bool isUnPackLikeOp(Operation *op) {
-  return isa<IREE::LinalgExt::UnsetEncodingOp, tensor::UnPackOp>(op);
+/// Returns true if the operation is an `unpack` op or an `unset_encoding` op,
+/// or an `extract_slice` op whose source operand matches those criteria,
+/// recursively.
+/// The idea is that we want to ensure that `extract_slice` ops can't prevent
+/// fusion between a `unset_encoding` producer and some linalg consumer. In
+///   %0 = unset_encoding ...
+///   %1 = extract_slice %0 ...
+///   %2 = linalg.generic ins(%1) ...
+/// we are not content to be fusing %1 into %0, we also want to be fusing %2,
+/// so we want to prevent %1 from acting as a consumer fusion barrier.
+static bool isUnpackLikeOpViaExtractSliceOps(Operation *op) {
+  if (isa<IREE::LinalgExt::UnsetEncodingOp, tensor::UnPackOp>(op)) {
+    return true;
+  }
+  if (isa<tensor::ExtractSliceOp>(op)) {
+    Value source = op->getOperand(0);
+    Operation *producer = source.getDefiningOp();
+    if (isUnpackLikeOpViaExtractSliceOps(producer)) {
+      return true;
+    }
+  }
+  return false;
 }
 
 /// Since `iree_linalg_ext.set_encoding` doesnt have padding semantics a
@@ -472,7 +489,7 @@
 
   // Fuse unset_encoding operations with `tensor.extract_slice` and elementwise
   // generic ops.
-  if (isUnPackLikeOp(producer)) {
+  if (isUnpackLikeOpViaExtractSliceOps(producer)) {
     // Fuse `unset_encoding` -> `extract_slice` op since they get folded into
     // `unpack` on materialization.
     if (isa<tensor::ExtractSliceOp>(consumer)) {
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/OptimizeNumerics.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/OptimizeNumerics.cpp
index b4b3ccf..471fb70 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/OptimizeNumerics.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/OptimizeNumerics.cpp
@@ -28,18 +28,18 @@
   }
 }
 
-Type withNewElementType(Type origType, Type elementType) {
-  if (auto st = llvm::dyn_cast<ShapedType>(origType)) {
+Type withNewElementType(Type originalType, Type elementType) {
+  if (auto st = llvm::dyn_cast<ShapedType>(originalType)) {
     return st.clone(elementType);
   } else {
     return elementType;
   }
 }
 
-Type makeLowPType(Type origType, int bitWidth) {
-  auto *context = origType.getContext();
+Type makeLowPType(Type originalType, int bitWidth) {
+  auto *context = originalType.getContext();
   auto elementType = IntegerType::get(context, bitWidth);
-  return withNewElementType(origType, elementType);
+  return withNewElementType(originalType, elementType);
 }
 
 Value castNumeric(Value origValue, Type toType, bool isSigned,
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td b/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td
index 3bab708..1c4864e 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td
@@ -250,11 +250,6 @@
 def SetEncoding : Pass<"iree-flow-set-encoding", ""> {
   let summary = "Introduce tensor encoding for compute operations";
   let constructor = "mlir::iree_compiler::IREE::Flow::createSetEncodingPass()";
-  let options = [
-    Option<"defaultPadding", "default-padding", "int64_t",
-           /*default=*/"16",
-           "Default padding to use so packing can be done without padding during the packing">
-  ];
 }
 
 def TensorPadToTensorInsertSlice :
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/SetEncoding.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/SetEncoding.cpp
index 0cc9c55..3d42025 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/SetEncoding.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/SetEncoding.cpp
@@ -21,8 +21,12 @@
 #include "mlir/Dialect/MemRef/Transforms/Transforms.h"
 #include "mlir/Dialect/Tensor/IR/Tensor.h"
 #include "mlir/Dialect/Tensor/Utils/Utils.h"
+#include "mlir/IR/BuiltinAttributeInterfaces.h"
+#include "mlir/IR/BuiltinAttributes.h"
 #include "mlir/IR/BuiltinTypes.h"
+#include "mlir/IR/OpDefinition.h"
 #include "mlir/IR/PatternMatch.h"
+#include "mlir/IR/Types.h"
 #include "mlir/Transforms/GreedyPatternRewriteDriver.h"
 
 namespace mlir {
@@ -30,85 +34,104 @@
 namespace IREE {
 namespace Flow {
 
-using IREE::LinalgExt::EncodingRole;
-using IREE::LinalgExt::EncodingUser;
-
 //===---------------------------------------------------------------------===//
 // Utility functions
 //===---------------------------------------------------------------------===//
 
-// Returns the element type of `t` if it is a `ShapedType`, else return
-// `t` itself.
-static Type getElementTypeOrType(Type t) {
-  if (auto shapedType = llvm::dyn_cast<ShapedType>(t)) {
-    return shapedType.getElementType();
+/// Pads `value` enough for any actual tile sizes that could result from
+/// materialization of `encodingAttr`.
+static Value pad(OpBuilder &builder, Location loc, Value source,
+                 IREE::LinalgExt::EncodingAttr encodingAttr) {
+  RankedTensorType sourceType = source.getType().cast<RankedTensorType>();
+  Type elemType = sourceType.getElementType();
+  size_t rank = sourceType.getRank();
+  RankedTensorType tensorTypeWithEncoding =
+      RankedTensorType::get(sourceType.getShape(), elemType, encodingAttr);
+  SmallVector<OpFoldResult> lowPad(rank, builder.getIndexAttr(0));
+  SmallVector<Type> resultTypes(rank, builder.getIndexType());
+
+  ValueRange encodingPaddingSizes =
+      builder
+          .create<IREE::LinalgExt::UpperBoundTileSizeOp>(
+              loc, resultTypes, TypeAttr::get(tensorTypeWithEncoding))
+          .getResults();
+  SmallVector<OpFoldResult> highPad(rank);
+  AffineExpr tileExpr, shapeExpr;
+  bindSymbols(builder.getContext(), tileExpr, shapeExpr);
+  AffineExpr highPadExpr = shapeExpr.ceilDiv(tileExpr) * tileExpr - shapeExpr;
+  for (size_t i = 0; i < rank; ++i) {
+    highPad[i] = affine::makeComposedFoldedAffineApply(
+        builder, loc, highPadExpr,
+        getAsOpFoldResult({encodingPaddingSizes[i],
+                           builder.create<tensor::DimOp>(loc, source, i)}));
   }
-  return t;
+
+  Value zero = builder.create<arith::ConstantOp>(loc, elemType,
+                                                 builder.getZeroAttr(elemType));
+  return builder.create<tensor::PadOp>(loc, /*resultType=*/nullptr, source,
+                                       lowPad, highPad, zero);
 }
 
-/// Returns a constant 0 of type `elementType`.
-static FailureOr<Value> getZero(OpBuilder &builder, Location loc,
-                                Type elementType) {
-  TypedAttr zeroVal =
-      TypeSwitch<Type, TypedAttr>(elementType)
-          .Case<FloatType>([&](FloatType floatType) -> Attribute {
-            return cast<TypedAttr>(builder.getFloatAttr(floatType, 0));
-          })
-          .Case<IntegerType>([&](IntegerType intType) -> Attribute {
-            return cast<TypedAttr>(builder.getIntegerAttr(intType, 0));
-          })
-          .Default([](Type type) { return nullptr; });
-  if (!zeroVal)
-    return failure();
-  return builder.create<arith::ConstantOp>(loc, elementType, zeroVal)
-      .getResult();
+static Value setEncoding(OpBuilder &builder, Location loc, Value source,
+                         IREE::LinalgExt::EncodingAttr encodingAttr) {
+  auto sourceType = source.getType().cast<RankedTensorType>();
+  auto resultType = RankedTensorType::get(
+      sourceType.getShape(), sourceType.getElementType(), encodingAttr);
+  return builder.create<IREE::LinalgExt::SetEncodingOp>(loc, resultType,
+                                                        source);
+};
+
+static LinalgExt::EncodingAttr makeEncoding(OpBuilder &builder,
+                                            LinalgExt::EncodingUser user,
+                                            LinalgExt::EncodingRole role,
+                                            Type originalType) {
+  auto *context = builder.getContext();
+  auto userAttr = LinalgExt::EncodingUserAttr::get(context, user);
+  auto roleAttr = LinalgExt::EncodingRoleAttr::get(context, role);
+  auto originalTypeAttr =
+      originalType ? TypeAttr::get(originalType) : TypeAttr{};
+  return LinalgExt::EncodingAttr::get(context, userAttr, roleAttr,
+                                      originalTypeAttr);
 }
 
-/// Pads `value` to `padding` if needed. If no padding is specified,
-/// return `value` itself.
-static FailureOr<Value>
-padIfNeeded(OpBuilder &builder, Location loc, Value value,
-            std::optional<int64_t> padding = std::nullopt) {
-  if (!padding)
-    return value;
-
-  OpFoldResult paddingOfr = builder.getIndexAttr(padding.value());
-  FailureOr<SmallVector<OpFoldResult>> shape =
-      LinalgExt::getDims(builder, loc, value);
-  if (failed(shape)) {
-    return failure();
+static Value padAndSetEncoding(OpBuilder &builder, Location loc, Value source,
+                               LinalgExt::EncodingUser user,
+                               LinalgExt::EncodingRole role) {
+  // No need to specify original_type in the encoding poadded to pad(), because
+  // the operand there is the `source` tensor, so it will default to reading its
+  // original shape.
+  auto encodingForPad =
+      makeEncoding(builder, user, role, /*originalType=*/Type{});
+  Value padded = pad(builder, loc, source, encodingForPad);
+  // For setEncoding() below, we potentially need to specify an encoding with an
+  // explicit original_type, because the operand there is the padded tensor
+  // returned by pad() above, but we want setEncoding to be aware of the
+  // original source tensor shape, not the padded tensor shape. To limit IR
+  // verbosity, we only specify the original original_type when it differs from
+  // the tensor type that the encoding is applied to.
+  auto encodingForSetEncoding = encodingForPad;
+  if (padded.getType() != source.getType()) {
+    encodingForSetEncoding =
+        makeEncoding(builder, user, role, source.getType());
   }
+  return setEncoding(builder, loc, padded, encodingForSetEncoding);
+}
 
-  OpFoldResult zero = builder.getIndexAttr(0);
-  SmallVector<OpFoldResult> lowPad(shape->size(), zero);
-  SmallVector<OpFoldResult> highPad(shape->size(), zero);
-
-  // The low padding is always zero. The high padding is
-  // shape.ceildDiv(padding) - shape.
-  AffineExpr paddingExpr, shapeExpr;
-  bindSymbols(builder.getContext(), paddingExpr, shapeExpr);
-  AffineExpr highPadExpr =
-      shapeExpr.ceilDiv(paddingExpr) * paddingExpr - shapeExpr;
-  for (auto shape : llvm::enumerate(shape.value())) {
-    highPad[shape.index()] = affine::makeComposedFoldedAffineApply(
-        builder, loc, highPadExpr, {paddingOfr, shape.value()});
-  }
-
-  // If all high padding evaluate to 0, then nothing to do.
-  if (llvm::all_of(highPad, [](OpFoldResult ofr) {
-        return isConstantIntValue(ofr, 0);
-      })) {
-    return value;
-  }
-
-  FailureOr<Value> zeroVal =
-      getZero(builder, loc, getElementTypeOrSelf(value.getType()));
-  if (failed(zeroVal)) {
-    return failure();
-  }
-  auto padOp = builder.create<tensor::PadOp>(loc, /*resultType=*/nullptr, value,
-                                             lowPad, highPad, zeroVal.value());
-  return padOp.getResult();
+static Value unsetEncodingAndExtractSlice(OpBuilder &builder, Location loc,
+                                          Value source,
+                                          SmallVector<OpFoldResult> sizes) {
+  auto sourceType = source.getType().cast<RankedTensorType>();
+  auto unsetEncodingReturnType =
+      RankedTensorType::get(sourceType.getShape(), sourceType.getElementType());
+  auto unsetEncoding = builder
+                           .create<IREE::LinalgExt::UnsetEncodingOp>(
+                               loc, unsetEncodingReturnType, source)
+                           .getResult();
+  auto rank = sourceType.getRank();
+  SmallVector<OpFoldResult> offsets(rank, builder.getIndexAttr(0));
+  SmallVector<OpFoldResult> strides(rank, builder.getIndexAttr(1));
+  return builder.create<tensor::ExtractSliceOp>(loc, unsetEncoding, offsets,
+                                                sizes, strides);
 }
 
 namespace {
@@ -116,10 +139,8 @@
 /// Rewrites the matmul op to work on tensors with encoding. Optionally
 /// also pads the operands.
 struct SetMatmulEncoding : public OpRewritePattern<linalg::MatmulOp> {
-  SetMatmulEncoding(MLIRContext *context, int64_t padding,
-                    PatternBenefit benefit = 1)
-      : OpRewritePattern<linalg::MatmulOp>(context, benefit), padding(padding) {
-  }
+  SetMatmulEncoding(MLIRContext *context, PatternBenefit benefit = 1)
+      : OpRewritePattern<linalg::MatmulOp>(context, benefit) {}
 
   LogicalResult matchAndRewrite(linalg::MatmulOp matmulOp,
                                 PatternRewriter &rewriter) const override {
@@ -154,26 +175,26 @@
       return failure();
     }
 
-    EncodingUser user;
+    LinalgExt::EncodingUser user;
 
     if (lhsElemType.isF32() && rhsElemType.isF32() && outElemType.isF32()) {
-      user = EncodingUser::MATMUL_F32F32F32;
+      user = LinalgExt::EncodingUser::MATMUL_F32F32F32;
     } else if (lhsElemType.isF16() && rhsElemType.isF16() &&
                outElemType.isF32()) {
-      user = EncodingUser::MATMUL_F16F16F32;
+      user = LinalgExt::EncodingUser::MATMUL_F16F16F32;
     } else if (lhsElemType.isF16() && rhsElemType.isF16() &&
                outElemType.isF16()) {
-      user = EncodingUser::MATMUL_F16F16F16;
+      user = LinalgExt::EncodingUser::MATMUL_F16F16F16;
     } else if (lhsElemType.isBF16() && rhsElemType.isBF16() &&
                outElemType.isF32()) {
-      user = EncodingUser::MATMUL_BF16BF16F32;
+      user = LinalgExt::EncodingUser::MATMUL_BF16BF16F32;
     } else if (lhsElemType.isBF16() && rhsElemType.isBF16() &&
                outElemType.isBF16()) {
-      user = EncodingUser::MATMUL_BF16BF16BF16;
+      user = LinalgExt::EncodingUser::MATMUL_BF16BF16BF16;
     } else if (lhsElemType.isSignlessInteger(8) &&
                rhsElemType.isSignlessInteger(8) &&
                outElemType.isSignlessInteger(32)) {
-      user = EncodingUser::MATMUL_I8I8I32;
+      user = LinalgExt::EncodingUser::MATMUL_I8I8I32;
     } else {
       return rewriter.notifyMatchFailure(
           matmulOp,
@@ -182,79 +203,33 @@
 
     Location loc = matmulOp.getLoc();
 
-    // Set encoding for LHS (pad if necessary)
-    FailureOr<Value> paddedLhs = padIfNeeded(rewriter, loc, origLhs, padding);
-    if (failed(paddedLhs)) {
-      return rewriter.notifyMatchFailure(matmulOp, "failed to pad lhs");
+    Value encodedLhs = padAndSetEncoding(rewriter, loc, origLhs, user,
+                                         LinalgExt::EncodingRole::LHS);
+    Value encodedRhs = padAndSetEncoding(rewriter, loc, origRhs, user,
+                                         LinalgExt::EncodingRole::RHS);
+    Value encodedOut = padAndSetEncoding(rewriter, loc, origOut, user,
+                                         LinalgExt::EncodingRole::RESULT);
+
+    Value matmulTiled = rewriter
+                            .create<linalg::MatmulOp>(
+                                loc, encodedOut.getType(),
+                                ValueRange{encodedLhs, encodedRhs}, encodedOut)
+                            .getResult(0);
+
+    // Sizes are computed by original output size.
+    FailureOr<SmallVector<OpFoldResult>> origOutSizes =
+        LinalgExt::getDims(rewriter, loc, origOut);
+    if (failed(origOutSizes)) {
+      return rewriter.notifyMatchFailure(matmulOp,
+                                         "failed to get shape of result");
     }
 
-    // Set encoding for RHS (pad if necessary)
-    FailureOr<Value> paddedRhs = padIfNeeded(rewriter, loc, origRhs, padding);
-    if (failed(paddedRhs)) {
-      return rewriter.notifyMatchFailure(matmulOp, "failed to pad rhs");
-    }
+    Value result = unsetEncodingAndExtractSlice(rewriter, loc, matmulTiled,
+                                                origOutSizes.value());
 
-    // Set encoding for OUTS (pad if necessary)
-    FailureOr<Value> paddedOut = padIfNeeded(rewriter, loc, origOut, padding);
-    if (failed(paddedOut)) {
-      return rewriter.notifyMatchFailure(matmulOp, "failed to pad output");
-    }
-
-    auto createSetEncodingOp = [&](Value source, EncodingRole role) {
-      auto *context = rewriter.getContext();
-      auto userAttr = LinalgExt::EncodingUserAttr::get(context, user);
-      auto roleAttr = LinalgExt::EncodingRoleAttr::get(context, role);
-      auto encodingAttr =
-          LinalgExt::EncodingAttr::get(context, userAttr, roleAttr);
-      auto sourceType = source.getType().cast<RankedTensorType>();
-      auto resultType = RankedTensorType::get(
-          sourceType.getShape(), sourceType.getElementType(), encodingAttr);
-      return rewriter.create<IREE::LinalgExt::SetEncodingOp>(loc, resultType,
-                                                             source);
-    };
-
-    Value encodedLhs =
-        createSetEncodingOp(paddedLhs.value(), EncodingRole::LHS);
-    Value encodedRhs =
-        createSetEncodingOp(paddedRhs.value(), EncodingRole::RHS);
-    Value encodedOut =
-        createSetEncodingOp(paddedOut.value(), EncodingRole::RESULT);
-
-    auto matmulTiled = rewriter.create<linalg::MatmulOp>(
-        loc, encodedOut.getType(), ValueRange{encodedLhs, encodedRhs},
-        encodedOut);
-    auto unsetEncoding = rewriter.create<IREE::LinalgExt::UnsetEncodingOp>(
-        loc, paddedOut->getType(), matmulTiled.getResult(0));
-
-    Value replacement = unsetEncoding.getResult();
-    // If the output was padded, extract the actual output.
-    if (paddedOut.value() != origOut) {
-      auto replacementRank =
-          llvm::cast<RankedTensorType>(replacement.getType()).getRank();
-      // Offsets are all 0.
-      OpFoldResult zero = rewriter.getIndexAttr(0);
-      SmallVector<OpFoldResult> offsets(replacementRank, zero);
-      // Strides are all 1.
-      OpFoldResult one = rewriter.getIndexAttr(1);
-      SmallVector<OpFoldResult> strides(replacementRank, one);
-
-      // Sizes are computed by original output size.
-      FailureOr<SmallVector<OpFoldResult>> sizes =
-          LinalgExt::getDims(rewriter, loc, origOut);
-      if (failed(sizes)) {
-        return rewriter.notifyMatchFailure(matmulOp,
-                                           "failed to get shape of result");
-      }
-      replacement = rewriter.create<tensor::ExtractSliceOp>(
-          loc, replacement, offsets, sizes.value(), strides);
-    }
-
-    rewriter.replaceOp(matmulOp, replacement);
+    rewriter.replaceOp(matmulOp, result);
     return success();
   }
-
-private:
-  int64_t padding;
 };
 
 /// Pattern to fold a `linalg.fill` -> `iree_linalg_ext.set_encoding`
@@ -296,7 +271,7 @@
   MLIRContext *context = &getContext();
   {
     RewritePatternSet patterns(context);
-    patterns.insert<SetMatmulEncoding>(context, defaultPadding);
+    patterns.insert<SetMatmulEncoding>(context);
     linalg::FillOp::getCanonicalizationPatterns(patterns, context);
     patterns.insert<FoldFillWithSetEncoding>(context);
     memref::populateResolveRankedShapedTypeResultDimsPatterns(patterns);
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/form_dispatch_regions.mlir b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/form_dispatch_regions.mlir
index 6f24047..d9c65d3 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/form_dispatch_regions.mlir
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/form_dispatch_regions.mlir
@@ -288,13 +288,10 @@
 //  CHECK-SAME:     %[[ARG1:.+]]: tensor<?xf32>
 //       CHECK:   %[[RESULT0:.+]] = flow.dispatch.region
 //       CHECK:     %[[UNSET_ENCODING:.+]] = iree_linalg_ext.unset_encoding %[[ARG0]]
-//       CHECK:     %[[SLICE:.+]] = tensor.extract_slice
-//       CHECK:     flow.return %[[SLICE]]
-//       CHECK:   %[[RESULT1:.+]] = flow.dispatch.region
-//       CHECK:     %[[GENERIC:.+]] = linalg.generic
-//  CHECK-SAME:         ins(%[[RESULT0]], %[[ARG1]]
+//       CHECK:     %[[SLICE:.+]] = tensor.extract_slice %[[UNSET_ENCODING]]
+//       CHECK:     %[[GENERIC:.+]] = linalg.generic {{.*}} ins(%[[SLICE]]
 //       CHECK:     flow.return %[[GENERIC]]
-//       CHECK:   return %[[RESULT1]]
+//       CHECK:   return %[[RESULT0]]
 
 // -----
 
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/set_encoding.mlir b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/set_encoding.mlir
index 1096f42..cb2b32c 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/set_encoding.mlir
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/set_encoding.mlir
@@ -1,52 +1,40 @@
 // RUN: iree-opt --iree-flow-set-encoding --cse --split-input-file %s | FileCheck %s
-// RUN: iree-opt --pass-pipeline="builtin.module(func.func(iree-flow-set-encoding{default-padding=4}, cse))" --split-input-file %s | FileCheck %s --check-prefix=PADDING
 
-func.func @matmul_no_padding(%arg0 : tensor<128x256xf32>, %arg1 : tensor<256x512xf32>,
-    %arg2 : tensor<128x512xf32>) -> tensor<128x512xf32> {
-  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<128x256xf32>, tensor<256x512xf32>)
-      outs(%arg2 : tensor<128x512xf32>) -> tensor<128x512xf32>
-  return %0 : tensor<128x512xf32>
-}
-//      CHECK: func @matmul_no_padding(
-// CHECK-SAME:     %[[ARG0:.+]]: tensor<128x256xf32>
-// CHECK-SAME:     %[[ARG1:.+]]: tensor<256x512xf32>
-// CHECK-SAME:     %[[ARG2:.+]]: tensor<128x512xf32>
-//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[ARG0]]
-// CHECK-SAME:       tensor<128x256xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>
-//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[ARG1]]
-// CHECK-SAME:       tensor<256x512xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>
-//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// CHECK-SAME:       tensor<128x512xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
-//      CHECK:   %[[MATMUL:.+]] = linalg.matmul
-// CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
-// CHECK-SAME:       outs(%[[OUTS]] :
-//      CHECK:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
-//      CHECK:   return %[[RESULT]]
-
-// -----
-
-func.func @matmul_padding(%arg0 : tensor<100x250xf32>, %arg1 : tensor<250x500xf32>,
+func.func @matmul_f32f32f32(%arg0 : tensor<100x250xf32>, %arg1 : tensor<250x500xf32>,
     %arg2 : tensor<100x500xf32>) -> tensor<100x500xf32> {
   %0 = linalg.matmul ins(%arg0, %arg1 : tensor<100x250xf32>, tensor<250x500xf32>)
       outs(%arg2 : tensor<100x500xf32>) -> tensor<100x500xf32>
   return %0 : tensor<100x500xf32>
 }
-//      CHECK: func @matmul_padding(
+//      CHECK: #[[MAP:.+]] = affine_map<()[s0, s1] -> (-s1 + (s1 ceildiv s0) * s0)>
+//      CHECK: func @matmul_f32f32f32(
 // CHECK-SAME:     %[[ARG0:.+]]: tensor<100x250xf32>
 // CHECK-SAME:     %[[ARG1:.+]]: tensor<250x500xf32>
 // CHECK-SAME:     %[[ARG2:.+]]: tensor<100x500xf32>
-//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high[12, 6]
-//      CHECK:       tensor<100x250xf32> to tensor<112x256xf32>
-//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high[6, 12]
-//      CHECK:       tensor<250x500xf32> to tensor<256x512xf32>
-//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high[12, 12]
-//      CHECK:       tensor<100x500xf32> to tensor<112x512xf32>
+//  CHECK-DAG:     %[[C100:.+]] = arith.constant 100 : index
+//  CHECK-DAG:     %[[C250:.+]] = arith.constant 250 : index
+//  CHECK-DAG:     %[[C500:.+]] = arith.constant 500 : index
+//      CHECK:   %[[LHS_TILE_SIZE:.+]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x250xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_PADDING_SIZE0:.+]] = affine.apply #[[MAP]]()[%[[LHS_TILE_SIZE]]#0, %[[C100]]]
+//      CHECK:   %[[LHS_PADDING_SIZE1:.+]] = affine.apply #[[MAP]]()[%[[LHS_TILE_SIZE]]#1, %[[C250]]]
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high[%[[LHS_PADDING_SIZE0]], %[[LHS_PADDING_SIZE1]]]
+//      CHECK:       tensor<100x250xf32> to tensor<?x?xf32>
 //      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
-// CHECK-SAME:       tensor<112x256xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>
+// CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS, original_type = tensor<100x250xf32>>>
+//      CHECK:   %[[RHS_TILE_SIZE:.+]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<250x500xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_PADDING_SIZE0:.+]] = affine.apply #[[MAP]]()[%[[RHS_TILE_SIZE]]#0, %[[C250]]]
+//      CHECK:   %[[RHS_PADDING_SIZE1:.+]] = affine.apply #[[MAP]]()[%[[RHS_TILE_SIZE]]#1, %[[C500]]]
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high[%[[RHS_PADDING_SIZE0]], %[[RHS_PADDING_SIZE1]]]
+//      CHECK:       tensor<250x500xf32> to tensor<?x?xf32>
 //      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
-// CHECK-SAME:       tensor<256x512xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>
+// CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS, original_type = tensor<250x500xf32>>>
+//      CHECK:   %[[OUTS_TILE_SIZE:.+]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x500xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_PADDING_SIZE0:.+]] = affine.apply #[[MAP]]()[%[[OUTS_TILE_SIZE]]#0, %[[C100]]]
+//      CHECK:   %[[OUTS_PADDING_SIZE1:.+]] = affine.apply #[[MAP]]()[%[[OUTS_TILE_SIZE]]#1, %[[C500]]]
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high[%[[OUTS_PADDING_SIZE0]], %[[OUTS_PADDING_SIZE1]]]
+//      CHECK:       tensor<100x500xf32> to tensor<?x?xf32>
 //      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
-// CHECK-SAME:       tensor<112x512xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
+// CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT, original_type = tensor<100x500xf32>>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
@@ -54,187 +42,221 @@
 //      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [100, 500] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
-//      PADDING: func @matmul_padding(
-// PADDING-SAME:     %[[ARG0:.+]]: tensor<100x250xf32>
-// PADDING-SAME:     %[[ARG1:.+]]: tensor<250x500xf32>
-// PADDING-SAME:     %[[ARG2:.+]]: tensor<100x500xf32>
-//      PADDING:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high[0, 2]
-//      PADDING:       tensor<100x250xf32> to tensor<100x252xf32>
-//      PADDING:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high[2, 0]
-//      PADDING:       tensor<250x500xf32> to tensor<252x500xf32>
-//      PADDING:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
-// PADDING-SAME:       tensor<100x252xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>
-//      PADDING:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
-// PADDING-SAME:       tensor<252x500xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>
-//      PADDING:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// PADDING-SAME:       tensor<100x500xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
-//      PADDING:   %[[MATMUL:.+]] = linalg.matmul
-// PADDING-SAME:       ins(%[[LHS]], %[[RHS]] :
-// PADDING-SAME:       outs(%[[OUTS]] :
-//      PADDING:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
-//      PADDING:   return %[[RESULT]]
-
 // -----
 
-func.func @matmul_dynamic(%arg0 : tensor<?x?xf32>, %arg1 : tensor<?x?xf32>,
+func.func @matmul_f32f32f32_dynamic(%arg0 : tensor<?x?xf32>, %arg1 : tensor<?x?xf32>,
     %arg2 : tensor<?x?xf32>) -> tensor<?x?xf32> {
   %0 = linalg.matmul ins(%arg0, %arg1 : tensor<?x?xf32>, tensor<?x?xf32>)
       outs(%arg2 : tensor<?x?xf32>) -> tensor<?x?xf32>
   return %0 : tensor<?x?xf32>
 }
-//      CHECK: #[[MAP:.+]] = affine_map<()[s0] -> (-s0 + (s0 ceildiv 16) * 16)
-//      CHECK: func @matmul_dynamic(
-// CHECK-SAME:     %[[ARG0:[a-zA-Z0-9]+]]: tensor<?x?xf32>
-// CHECK-SAME:     %[[ARG1:[a-zA-Z0-9]+]]: tensor<?x?xf32>
-// CHECK-SAME:     %[[ARG2:[a-zA-Z0-9]+]]: tensor<?x?xf32>
-//  CHECK-DAG:   %[[C0:.+]] = arith.constant 0 : index
-//  CHECK-DAG:   %[[C1:.+]] = arith.constant 1 : index
-//  CHECK-DAG:   %[[LHS_D0:.+]] = tensor.dim %[[ARG0]], %[[C0]]
-//  CHECK-DAG:   %[[LHS_D1:.+]] = tensor.dim %[[ARG0]], %[[C1]]
-//  CHECK-DAG:   %[[HIGHPAD_LHS_0:.+]] = affine.apply #[[MAP]]()[%[[LHS_D0]]]
-//  CHECK-DAG:   %[[HIGHPAD_LHS_1:.+]] = affine.apply #[[MAP]]()[%[[LHS_D1]]]
-//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high[%[[HIGHPAD_LHS_0]], %[[HIGHPAD_LHS_1]]]
-//  CHECK-DAG:   %[[RHS_D0:.+]] = tensor.dim %[[ARG1]], %[[C0]]
-//  CHECK-DAG:   %[[RHS_D1:.+]] = tensor.dim %[[ARG1]], %[[C1]]
-//  CHECK-DAG:   %[[HIGHPAD_RHS_0:.+]] = affine.apply #[[MAP]]()[%[[RHS_D0]]]
-//  CHECK-DAG:   %[[HIGHPAD_RHS_1:.+]] = affine.apply #[[MAP]]()[%[[RHS_D1]]]
-//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high[%[[HIGHPAD_RHS_0]], %[[HIGHPAD_RHS_1]]]
-//  CHECK-DAG:   %[[OUTS_D0:.+]] = tensor.dim %[[ARG2]], %[[C0]]
-//  CHECK-DAG:   %[[OUTS_D1:.+]] = tensor.dim %[[ARG2]], %[[C1]]
-//  CHECK-DAG:   %[[HIGHPAD_OUTS_0:.+]] = affine.apply #[[MAP]]()[%[[OUTS_D0]]]
-//  CHECK-DAG:   %[[HIGHPAD_OUTS_1:.+]] = affine.apply #[[MAP]]()[%[[OUTS_D1]]]
-//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high[%[[HIGHPAD_OUTS_0]], %[[HIGHPAD_OUTS_1]]]
+//      CHECK: #[[MAP:.+]] = affine_map<()[s0, s1] -> (-s1 + (s1 ceildiv s0) * s0)>
+//      CHECK: func @matmul_f32f32f32_dynamic(
+// CHECK-SAME:     %[[ARG0:.+]]: tensor<?x?xf32>, %[[ARG1:.+]]: tensor<?x?xf32>, %[[ARG2:.+]]: tensor<?x?xf32>
+//  CHECK-DAG:     %[[C0:.+]] = arith.constant 0 : index
+//  CHECK-DAG:     %[[C1:.+]] = arith.constant 1 : index
+//      CHECK:   %[[LHS_TILE_SIZE:.+]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_DIM0:.+]] = tensor.dim %[[ARG0]], %[[C0]]
+//      CHECK:   %[[LHS_PADDING_SIZE0:.+]] = affine.apply #[[MAP]]()[%[[LHS_TILE_SIZE]]#0, %[[LHS_DIM0]]]
+//      CHECK:   %[[LHS_DIM1:.+]] = tensor.dim %[[ARG0]], %[[C1]]
+//      CHECK:   %[[LHS_PADDING_SIZE1:.+]] = affine.apply #[[MAP]]()[%[[LHS_TILE_SIZE]]#1, %[[LHS_DIM1]]]
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high[%[[LHS_PADDING_SIZE0]], %[[LHS_PADDING_SIZE1]]]
+//      CHECK:       tensor<?x?xf32> to tensor<?x?xf32>
 //      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
 // CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = LHS>>
+//      CHECK:   %[[RHS_TILE_SIZE:.+]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_DIM0:.+]] = tensor.dim %[[ARG1]], %[[C0]]
+//      CHECK:   %[[RHS_PADDING_SIZE0:.+]] = affine.apply #[[MAP]]()[%[[RHS_TILE_SIZE]]#0, %[[RHS_DIM0]]]
+//      CHECK:   %[[RHS_DIM1:.+]] = tensor.dim %[[ARG1]], %[[C1]]
+//      CHECK:   %[[RHS_PADDING_SIZE1:.+]] = affine.apply #[[MAP]]()[%[[RHS_TILE_SIZE]]#1, %[[RHS_DIM1]]]
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high[%[[RHS_PADDING_SIZE0]], %[[RHS_PADDING_SIZE1]]]
+//      CHECK:       tensor<?x?xf32> to tensor<?x?xf32>
 //      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
 // CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RHS>>
+//      CHECK:   %[[OUTS_TILE_SIZE:.+]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_DIM0:.+]] = tensor.dim %[[ARG2]], %[[C0]]
+//      CHECK:   %[[OUTS_PADDING_SIZE0:.+]] = affine.apply #[[MAP]]()[%[[OUTS_TILE_SIZE]]#0, %[[OUTS_DIM0]]]
+//      CHECK:   %[[OUTS_DIM1:.+]] = tensor.dim %[[ARG2]], %[[C1]]
+//      CHECK:   %[[OUTS_PADDING_SIZE1:.+]] = affine.apply #[[MAP]]()[%[[OUTS_TILE_SIZE]]#1, %[[OUTS_DIM1]]]
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high[%[[OUTS_PADDING_SIZE0]], %[[OUTS_PADDING_SIZE1]]]
+//      CHECK:       tensor<?x?xf32> to tensor<?x?xf32>
 //      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
 // CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F32F32F32, role = RESULT>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
 //      CHECK:   %[[RESULT_PADDED:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
-//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [%[[OUTS_D0]], %[[OUTS_D1]]] [1, 1]
+//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [{{.*}}] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
 // -----
 
-func.func @matmul_i8i8i32(%arg0 : tensor<128x256xi8>, %arg1 : tensor<256x512xi8>,
-    %arg2 : tensor<128x512xi32>) -> tensor<128x512xi32> {
-  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<128x256xi8>, tensor<256x512xi8>)
-      outs(%arg2 : tensor<128x512xi32>) -> tensor<128x512xi32>
-  return %0 : tensor<128x512xi32>
+func.func @matmul_i8i8i32(%arg0 : tensor<100x250xi8>, %arg1 : tensor<250x500xi8>,
+    %arg2 : tensor<100x500xi32>) -> tensor<100x500xi32> {
+  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<100x250xi8>, tensor<250x500xi8>)
+      outs(%arg2 : tensor<100x500xi32>) -> tensor<100x500xi32>
+  return %0 : tensor<100x500xi32>
 }
 //      CHECK: func @matmul_i8i8i32(
-// CHECK-SAME:     %[[ARG0:.+]]: tensor<128x256xi8>
-// CHECK-SAME:     %[[ARG1:.+]]: tensor<256x512xi8>
-// CHECK-SAME:     %[[ARG2:.+]]: tensor<128x512xi32>
-//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[ARG0]]
-// CHECK-SAME:       tensor<128x256xi8, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = LHS>>
-//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[ARG1]]
-// CHECK-SAME:       tensor<256x512xi8, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = RHS>>
-//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// CHECK-SAME:       tensor<128x512xi32, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = RESULT>>
+// CHECK-SAME:     %[[ARG0:.+]]: tensor<100x250xi8>
+// CHECK-SAME:     %[[ARG1:.+]]: tensor<250x500xi8>
+// CHECK-SAME:     %[[ARG2:.+]]: tensor<100x500xi32>
+//      CHECK:   %[[LHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x250xi8, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high
+//      CHECK:       tensor<100x250xi8> to tensor<?x?xi8>
+//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
+// CHECK-SAME:       tensor<?x?xi8, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = LHS, original_type = tensor<100x250xi8>>>
+//      CHECK:   %[[RHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<250x500xi8, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high
+//      CHECK:       tensor<250x500xi8> to tensor<?x?xi8>
+//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
+// CHECK-SAME:       tensor<?x?xi8, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = RHS, original_type = tensor<250x500xi8>>>
+//      CHECK:   %[[OUTS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x500xi32, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high
+//      CHECK:       tensor<100x500xi32> to tensor<?x?xi32>
+//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
+// CHECK-SAME:       tensor<?x?xi32, #iree_linalg_ext.encoding<user = MATMUL_I8I8I32, role = RESULT, original_type = tensor<100x500xi32>>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
-//      CHECK:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT_PADDED:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [100, 500] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
 // -----
 
-func.func @matmul_f16f16f32(%arg0 : tensor<128x256xf16>, %arg1 : tensor<256x512xf16>,
-    %arg2 : tensor<128x512xf32>) -> tensor<128x512xf32> {
-  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<128x256xf16>, tensor<256x512xf16>)
-      outs(%arg2 : tensor<128x512xf32>) -> tensor<128x512xf32>
-  return %0 : tensor<128x512xf32>
+func.func @matmul_f16f16f32(%arg0 : tensor<100x250xf16>, %arg1 : tensor<250x500xf16>,
+    %arg2 : tensor<100x500xf32>) -> tensor<100x500xf32> {
+  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<100x250xf16>, tensor<250x500xf16>)
+      outs(%arg2 : tensor<100x500xf32>) -> tensor<100x500xf32>
+  return %0 : tensor<100x500xf32>
 }
 //      CHECK: func @matmul_f16f16f32(
-// CHECK-SAME:     %[[ARG0:.+]]: tensor<128x256xf16>
-// CHECK-SAME:     %[[ARG1:.+]]: tensor<256x512xf16>
-// CHECK-SAME:     %[[ARG2:.+]]: tensor<128x512xf32>
-//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[ARG0]]
-// CHECK-SAME:       tensor<128x256xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = LHS>>
-//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[ARG1]]
-// CHECK-SAME:       tensor<256x512xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = RHS>>
-//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// CHECK-SAME:       tensor<128x512xf32, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = RESULT>>
+// CHECK-SAME:     %[[ARG0:.+]]: tensor<100x250xf16>
+// CHECK-SAME:     %[[ARG1:.+]]: tensor<250x500xf16>
+// CHECK-SAME:     %[[ARG2:.+]]: tensor<100x500xf32>
+//      CHECK:   %[[LHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x250xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high
+//      CHECK:       tensor<100x250xf16> to tensor<?x?xf16>
+//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
+// CHECK-SAME:       tensor<?x?xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = LHS, original_type = tensor<100x250xf16>>>
+//      CHECK:   %[[RHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<250x500xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high
+//      CHECK:       tensor<250x500xf16> to tensor<?x?xf16>
+//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
+// CHECK-SAME:       tensor<?x?xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = RHS, original_type = tensor<250x500xf16>>>
+//      CHECK:   %[[OUTS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x500xf32, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high
+//      CHECK:       tensor<100x500xf32> to tensor<?x?xf32>
+//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
+// CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_F16F16F32, role = RESULT, original_type = tensor<100x500xf32>>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
-//      CHECK:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT_PADDED:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [100, 500] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
 // -----
 
-func.func @matmul_f16f16f16(%arg0 : tensor<128x256xf16>, %arg1 : tensor<256x512xf16>,
-    %arg2 : tensor<128x512xf16>) -> tensor<128x512xf16> {
-  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<128x256xf16>, tensor<256x512xf16>)
-      outs(%arg2 : tensor<128x512xf16>) -> tensor<128x512xf16>
-  return %0 : tensor<128x512xf16>
+func.func @matmul_f16f16f16(%arg0 : tensor<100x250xf16>, %arg1 : tensor<250x500xf16>,
+    %arg2 : tensor<100x500xf16>) -> tensor<100x500xf16> {
+  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<100x250xf16>, tensor<250x500xf16>)
+      outs(%arg2 : tensor<100x500xf16>) -> tensor<100x500xf16>
+  return %0 : tensor<100x500xf16>
 }
 //      CHECK: func @matmul_f16f16f16(
-// CHECK-SAME:     %[[ARG0:.+]]: tensor<128x256xf16>
-// CHECK-SAME:     %[[ARG1:.+]]: tensor<256x512xf16>
-// CHECK-SAME:     %[[ARG2:.+]]: tensor<128x512xf16>
-//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[ARG0]]
-// CHECK-SAME:       tensor<128x256xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = LHS>>
-//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[ARG1]]
-// CHECK-SAME:       tensor<256x512xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = RHS>>
-//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// CHECK-SAME:       tensor<128x512xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = RESULT>>
+// CHECK-SAME:     %[[ARG0:.+]]: tensor<100x250xf16>
+// CHECK-SAME:     %[[ARG1:.+]]: tensor<250x500xf16>
+// CHECK-SAME:     %[[ARG2:.+]]: tensor<100x500xf16>
+//      CHECK:   %[[LHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x250xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high
+//      CHECK:       tensor<100x250xf16> to tensor<?x?xf16>
+//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
+// CHECK-SAME:       tensor<?x?xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = LHS, original_type = tensor<100x250xf16>>>
+//      CHECK:   %[[RHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<250x500xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high
+//      CHECK:       tensor<250x500xf16> to tensor<?x?xf16>
+//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
+// CHECK-SAME:       tensor<?x?xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = RHS, original_type = tensor<250x500xf16>>>
+//      CHECK:   %[[OUTS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x500xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high
+//      CHECK:       tensor<100x500xf16> to tensor<?x?xf16>
+//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
+// CHECK-SAME:       tensor<?x?xf16, #iree_linalg_ext.encoding<user = MATMUL_F16F16F16, role = RESULT, original_type = tensor<100x500xf16>>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
-//      CHECK:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT_PADDED:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [100, 500] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
 // -----
 
-func.func @matmul_bf16bf16f32(%arg0 : tensor<128x256xbf16>, %arg1 : tensor<256x512xbf16>,
-    %arg2 : tensor<128x512xf32>) -> tensor<128x512xf32> {
-  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<128x256xbf16>, tensor<256x512xbf16>)
-      outs(%arg2 : tensor<128x512xf32>) -> tensor<128x512xf32>
-  return %0 : tensor<128x512xf32>
+func.func @matmul_bf16bf16f32(%arg0 : tensor<100x250xbf16>, %arg1 : tensor<250x500xbf16>,
+    %arg2 : tensor<100x500xf32>) -> tensor<100x500xf32> {
+  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<100x250xbf16>, tensor<250x500xbf16>)
+      outs(%arg2 : tensor<100x500xf32>) -> tensor<100x500xf32>
+  return %0 : tensor<100x500xf32>
 }
 //      CHECK: func @matmul_bf16bf16f32(
-// CHECK-SAME:     %[[ARG0:.+]]: tensor<128x256xbf16>
-// CHECK-SAME:     %[[ARG1:.+]]: tensor<256x512xbf16>
-// CHECK-SAME:     %[[ARG2:.+]]: tensor<128x512xf32>
-//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[ARG0]]
-// CHECK-SAME:       tensor<128x256xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = LHS>>
-//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[ARG1]]
-// CHECK-SAME:       tensor<256x512xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = RHS>>
-//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// CHECK-SAME:       tensor<128x512xf32, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = RESULT>>
+// CHECK-SAME:     %[[ARG0:.+]]: tensor<100x250xbf16>
+// CHECK-SAME:     %[[ARG1:.+]]: tensor<250x500xbf16>
+// CHECK-SAME:     %[[ARG2:.+]]: tensor<100x500xf32>
+//      CHECK:   %[[LHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x250xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high
+//      CHECK:       tensor<100x250xbf16> to tensor<?x?xbf16>
+//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
+// CHECK-SAME:       tensor<?x?xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = LHS, original_type = tensor<100x250xbf16>>>
+//      CHECK:   %[[RHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<250x500xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high
+//      CHECK:       tensor<250x500xbf16> to tensor<?x?xbf16>
+//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
+// CHECK-SAME:       tensor<?x?xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = RHS, original_type = tensor<250x500xbf16>>>
+//      CHECK:   %[[OUTS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x500xf32, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high
+//      CHECK:       tensor<100x500xf32> to tensor<?x?xf32>
+//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
+// CHECK-SAME:       tensor<?x?xf32, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16F32, role = RESULT, original_type = tensor<100x500xf32>>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
-//      CHECK:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT_PADDED:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [100, 500] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
 // -----
 
-func.func @matmul_bf16bf16bf16(%arg0 : tensor<128x256xbf16>, %arg1 : tensor<256x512xbf16>,
-    %arg2 : tensor<128x512xbf16>) -> tensor<128x512xbf16> {
-  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<128x256xbf16>, tensor<256x512xbf16>)
-      outs(%arg2 : tensor<128x512xbf16>) -> tensor<128x512xbf16>
-  return %0 : tensor<128x512xbf16>
+func.func @matmul_bf16bf16bf16(%arg0 : tensor<100x250xbf16>, %arg1 : tensor<250x500xbf16>,
+    %arg2 : tensor<100x500xbf16>) -> tensor<100x500xbf16> {
+  %0 = linalg.matmul ins(%arg0, %arg1 : tensor<100x250xbf16>, tensor<250x500xbf16>)
+      outs(%arg2 : tensor<100x500xbf16>) -> tensor<100x500xbf16>
+  return %0 : tensor<100x500xbf16>
 }
 //      CHECK: func @matmul_bf16bf16bf16(
-// CHECK-SAME:     %[[ARG0:.+]]: tensor<128x256xbf16>
-// CHECK-SAME:     %[[ARG1:.+]]: tensor<256x512xbf16>
-// CHECK-SAME:     %[[ARG2:.+]]: tensor<128x512xbf16>
-//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[ARG0]]
-// CHECK-SAME:       tensor<128x256xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = LHS>>
-//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[ARG1]]
-// CHECK-SAME:       tensor<256x512xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = RHS>>
-//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[ARG2]]
-// CHECK-SAME:       tensor<128x512xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = RESULT>>
+// CHECK-SAME:     %[[ARG0:.+]]: tensor<100x250xbf16>
+// CHECK-SAME:     %[[ARG1:.+]]: tensor<250x500xbf16>
+// CHECK-SAME:     %[[ARG2:.+]]: tensor<100x500xbf16>
+//      CHECK:   %[[LHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x250xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = LHS>> -> index, index
+//      CHECK:   %[[LHS_PAD:.+]] = tensor.pad %[[ARG0]] low[0, 0] high
+//      CHECK:       tensor<100x250xbf16> to tensor<?x?xbf16>
+//      CHECK:   %[[LHS:.+]] = iree_linalg_ext.set_encoding %[[LHS_PAD]]
+// CHECK-SAME:       tensor<?x?xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = LHS, original_type = tensor<100x250xbf16>>>
+//      CHECK:   %[[RHS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<250x500xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = RHS>> -> index, index
+//      CHECK:   %[[RHS_PAD:.+]] = tensor.pad %[[ARG1]] low[0, 0] high
+//      CHECK:       tensor<250x500xbf16> to tensor<?x?xbf16>
+//      CHECK:   %[[RHS:.+]] = iree_linalg_ext.set_encoding %[[RHS_PAD]]
+// CHECK-SAME:       tensor<?x?xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = RHS, original_type = tensor<250x500xbf16>>>
+//      CHECK:   %[[OUTS_TILE_SIZE]]:2 = iree_linalg_ext.upper_bound_tile_size tensor<100x500xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = RESULT>> -> index, index
+//      CHECK:   %[[OUTS_PAD:.+]] = tensor.pad %[[ARG2]] low[0, 0] high
+//      CHECK:       tensor<100x500xbf16> to tensor<?x?xbf16>
+//      CHECK:   %[[OUTS:.+]] = iree_linalg_ext.set_encoding %[[OUTS_PAD]]
+// CHECK-SAME:       tensor<?x?xbf16, #iree_linalg_ext.encoding<user = MATMUL_BF16BF16BF16, role = RESULT, original_type = tensor<100x500xbf16>>>
 //      CHECK:   %[[MATMUL:.+]] = linalg.matmul
 // CHECK-SAME:       ins(%[[LHS]], %[[RHS]] :
 // CHECK-SAME:       outs(%[[OUTS]] :
-//      CHECK:   %[[RESULT:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT_PADDED:.+]] = iree_linalg_ext.unset_encoding %[[MATMUL]]
+//      CHECK:   %[[RESULT:.+]] = tensor.extract_slice %[[RESULT_PADDED]][0, 0] [100, 500] [1, 1]
 //      CHECK:   return %[[RESULT]]
 
 // -----
diff --git a/compiler/src/iree/compiler/Dialect/HAL/IR/HALOps.cpp b/compiler/src/iree/compiler/Dialect/HAL/IR/HALOps.cpp
index 0ccf60e..d702e1d 100644
--- a/compiler/src/iree/compiler/Dialect/HAL/IR/HALOps.cpp
+++ b/compiler/src/iree/compiler/Dialect/HAL/IR/HALOps.cpp
@@ -665,9 +665,10 @@
 static LogicalResult verifyDeviceQueueFences(Operation *queueOp,
                                              Value waitFence,
                                              Value signalFence) {
-  if (waitFence == signalFence) {
+  if (waitFence == signalFence &&
+      !isa<IREE::Util::NullOp>(waitFence.getDefiningOp())) {
     return queueOp->emitOpError() << "device queue operations cannot wait and "
-                                     "signal on the same fence";
+                                     "signal on the same fence.";
   }
   return success();
 }
diff --git a/compiler/src/iree/compiler/Dialect/HAL/Transforms/BUILD.bazel b/compiler/src/iree/compiler/Dialect/HAL/Transforms/BUILD.bazel
index b109c08..f511949 100644
--- a/compiler/src/iree/compiler/Dialect/HAL/Transforms/BUILD.bazel
+++ b/compiler/src/iree/compiler/Dialect/HAL/Transforms/BUILD.bazel
@@ -40,6 +40,7 @@
         "Passes.h",
     ],
     deps = [
+        "//compiler/src/iree/compiler/Codegen/Common/CPU:CommonCPUPasses",
         "//compiler/src/iree/compiler/Codegen/Dialect:IREECodegenDialect",
         "//compiler/src/iree/compiler/Dialect/Flow/IR",
         "//compiler/src/iree/compiler/Dialect/HAL/Analysis",
diff --git a/compiler/src/iree/compiler/Dialect/HAL/Transforms/CMakeLists.txt b/compiler/src/iree/compiler/Dialect/HAL/Transforms/CMakeLists.txt
index bdf4010..703768b 100644
--- a/compiler/src/iree/compiler/Dialect/HAL/Transforms/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Dialect/HAL/Transforms/CMakeLists.txt
@@ -51,6 +51,7 @@
     MLIRSupport
     MLIRTensorDialect
     MLIRTransforms
+    iree::compiler::Codegen::Common::CPU::CommonCPUPasses
     iree::compiler::Codegen::Dialect::IREECodegenDialect
     iree::compiler::Dialect::Flow::IR
     iree::compiler::Dialect::HAL::Analysis
diff --git a/compiler/src/iree/compiler/Dialect/HAL/Transforms/Passes.cpp b/compiler/src/iree/compiler/Dialect/HAL/Transforms/Passes.cpp
index 917b9a6..8afc861 100644
--- a/compiler/src/iree/compiler/Dialect/HAL/Transforms/Passes.cpp
+++ b/compiler/src/iree/compiler/Dialect/HAL/Transforms/Passes.cpp
@@ -8,6 +8,7 @@
 
 #include <memory>
 
+#include "iree/compiler/Codegen/Common/CPU/Passes.h"
 #include "iree/compiler/Dialect/HAL/IR/HALDialect.h"
 #include "iree/compiler/Dialect/HAL/IR/HALOps.h"
 #include "iree/compiler/Dialect/Util/Transforms/Passes.h"
@@ -223,6 +224,9 @@
   // Executable translation
   //----------------------------------------------------------------------------
 
+  FunctionLikeNest(passManager)
+      .addPass(createCPUMaterializeUpperBoundTileSizePass);
+
   // Preprocess executables using an external tool. The tool may mutate one or
   // more variants and even insert or remove variants.
   for (auto command : clPreprocessExecutablesWith) {
diff --git a/compiler/src/iree/compiler/Dialect/VMVX/Transforms/BUILD.bazel b/compiler/src/iree/compiler/Dialect/VMVX/Transforms/BUILD.bazel
index 3e53b51..488ebf2 100644
--- a/compiler/src/iree/compiler/Dialect/VMVX/Transforms/BUILD.bazel
+++ b/compiler/src/iree/compiler/Dialect/VMVX/Transforms/BUILD.bazel
@@ -51,6 +51,7 @@
     deps = [
         ":PassHeaders",
         "//compiler/src/iree/compiler/Codegen/Common",
+        "//compiler/src/iree/compiler/Codegen/Common/CPU:CommonCPUPasses",
         "//compiler/src/iree/compiler/Codegen/LLVMCPU",
         "//compiler/src/iree/compiler/Codegen/Transforms",
         "//compiler/src/iree/compiler/Codegen/Utils",
diff --git a/compiler/src/iree/compiler/Dialect/VMVX/Transforms/CMakeLists.txt b/compiler/src/iree/compiler/Dialect/VMVX/Transforms/CMakeLists.txt
index 64c3449..8825a46 100644
--- a/compiler/src/iree/compiler/Dialect/VMVX/Transforms/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Dialect/VMVX/Transforms/CMakeLists.txt
@@ -73,6 +73,7 @@
     MLIRVectorDialect
     MLIRVectorToSCF
     iree::compiler::Codegen::Common
+    iree::compiler::Codegen::Common::CPU::CommonCPUPasses
     iree::compiler::Codegen::LLVMCPU
     iree::compiler::Codegen::Transforms
     iree::compiler::Codegen::Utils
diff --git a/compiler/src/iree/compiler/Dialect/VMVX/Transforms/Passes.cpp b/compiler/src/iree/compiler/Dialect/VMVX/Transforms/Passes.cpp
index fe7d510..ca89276 100644
--- a/compiler/src/iree/compiler/Dialect/VMVX/Transforms/Passes.cpp
+++ b/compiler/src/iree/compiler/Dialect/VMVX/Transforms/Passes.cpp
@@ -9,6 +9,7 @@
 #include <memory>
 
 #include "iree-dialects/Dialect/LinalgExt/Passes/Passes.h"
+#include "iree/compiler/Codegen/Common/CPU/Passes.h"
 #include "iree/compiler/Codegen/Common/Passes.h"
 #include "iree/compiler/Codegen/LLVMCPU/Passes.h"
 #include "iree/compiler/Codegen/VMVX/Passes.h"
@@ -35,7 +36,7 @@
   // ---------------------------------------------------------------------------
   addCommonTargetExecutablePreprocessingPasses(passManager.nest<ModuleOp>());
   passManager.nest<ModuleOp>().addNestedPass<func::FuncOp>(
-      createVMVXMaterializeEncodingPass());
+      createCPUMaterializeEncodingPass());
   // TODO: Remove the following pass the plumb support for #hal.descriptor_type
   // memory space through the stack.
   passManager.nest<ModuleOp>().addNestedPass<func::FuncOp>(
diff --git a/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtBase.td b/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtBase.td
index 2c3e8a7..2ce3eeb 100644
--- a/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtBase.td
+++ b/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtBase.td
@@ -106,17 +106,18 @@
   }];
 
   let assemblyFormat = [{
-    `<` `user` `=` $user
-      `,` `role` `=` $role `>`
+    `<` `user` `=` `` $user
+    `,` `role` `=` `` $role
+    (`,` `original_type` `=` $originalType^)? `>`
   }];
 
   let parameters = (ins
     AttrParameter<"EncodingUserAttr", "kind of operation using this tensor">:$user,
-    AttrParameter<"EncodingRoleAttr", "role of this tensor as an operand">:$role
+    AttrParameter<"EncodingRoleAttr", "role of this tensor as an operand">:$role,
+    OptionalParameter<"TypeAttr", "type of the original tensor type before padding">:$originalType
   );
 
   let genVerifyDecl = 0;
 }
 
-
 #endif // IREE_DIALECT_LINALGEXT_BASE
diff --git a/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtOps.td b/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtOps.td
index 60a0b27..2d6ba82 100644
--- a/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtOps.td
+++ b/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/IR/LinalgExtOps.td
@@ -1072,6 +1072,25 @@
   }];
 }
 
+def IREELinalgExt_UpperBoundTileSizeOp : IREELinalgExt_PureOp<"upper_bound_tile_size",
+    [Pure]> {
+  let summary = "returns an upper bound on tile sizes";
+  let description = [{
+    This returns the largest tile sizes that might result from materialization
+    of the given encoding. This can be used outside of target-specific code, so
+    there may be multiple targets, and this will return the maximum tile size
+    from iterating over all of them. The evaluation happens in the
+    MaterializeUpperBoundTileSize pass.
+  }];
+
+  let arguments = (ins TypeAttrOf<AnyRankedTensor>:$tensorType);
+  let results = (outs Variadic<Index>:$results);
+
+  let assemblyFormat = [{
+    attr-dict $tensorType `->` type($results)
+  }];
+}
+
 def IREELinalgExt_UnsetEncodingOp : IREELinalgExt_PureOp<"unset_encoding", [
     DeclareOpInterfaceMethods<ReifyRankedShapedTypeOpInterface>, Pure
   ]> {
diff --git a/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/Passes/Passes.h b/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/Passes/Passes.h
index 2005bbe..d6d13f4 100644
--- a/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/Passes/Passes.h
+++ b/llvm-external-projects/iree-dialects/include/iree-dialects/Dialect/LinalgExt/Passes/Passes.h
@@ -131,6 +131,9 @@
     MaterializeEncodingTypeConverter &typeConverter,
     MaterializeEncodingValueFn materializeEncodingValueFn = {});
 
+void populateMaterializeUpperBoundTileSizePatterns(
+    RewritePatternSet &patterns, MaterializeEncodingFn materializeEncodingFn);
+
 /// Pass to apply patterns specified by `populateMaterializeEncodingPass`.
 std::unique_ptr<OperationPass<func::FuncOp>> createMaterializeEncodingPass();
 
diff --git a/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/MaterializeEncoding.cpp b/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/MaterializeEncoding.cpp
index 30dcdcd..3172ebe 100644
--- a/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/MaterializeEncoding.cpp
+++ b/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/MaterializeEncoding.cpp
@@ -9,13 +9,19 @@
 #include "iree-dialects/Dialect/LinalgExt/Passes/Passes.h"
 #include "iree-dialects/Dialect/LinalgExt/Utils/Utils.h"
 #include "mlir/Dialect/Affine/IR/AffineOps.h"
+#include "mlir/Dialect/Arith/IR/Arith.h"
 #include "mlir/Dialect/Linalg/IR/Linalg.h"
 #include "mlir/Dialect/MemRef/Transforms/Transforms.h"
 #include "mlir/Dialect/Tensor/IR/Tensor.h"
 #include "mlir/Dialect/Tensor/Transforms/Transforms.h"
+#include "mlir/IR/BuiltinAttributes.h"
+#include "mlir/IR/BuiltinTypeInterfaces.h"
+#include "mlir/IR/BuiltinTypes.h"
+#include "mlir/IR/OpDefinition.h"
 #include "mlir/IR/PatternMatch.h"
 #include "mlir/Transforms/DialectConversion.h"
 #include "mlir/Transforms/GreedyPatternRewriteDriver.h"
+#include "llvm/ADT/STLExtras.h"
 #include "llvm/ADT/TypeSwitch.h"
 
 using namespace mlir;
@@ -30,21 +36,34 @@
   return type.getEncoding().dyn_cast_or_null<EncodingAttr>();
 }
 
+static RankedTensorType getOriginalTypeWithEncoding(RankedTensorType type) {
+  auto encoding = getEncodingAttr(type);
+  if (!encoding) {
+    return type;
+  }
+  RankedTensorType originalType = type;
+  if (auto originalTypeAttr = encoding.getOriginalType()) {
+    originalType = originalTypeAttr.getValue().cast<RankedTensorType>();
+  }
+  return RankedTensorType::get(originalType.getShape(),
+                               originalType.getElementType(), encoding);
+}
+
 /// For a given tensor type with an encoding, return the materialized
 /// type to use for it. If no encoding is set, then return the tensor type
 /// itself.
 static RankedTensorType
 getMaterializedType(RankedTensorType tensorType,
                     MaterializeEncodingFn materializeEncodingFn) {
-  if (!getEncodingAttr(tensorType))
-    return tensorType;
+
   FailureOr<MaterializeEncodingInfo> materializeEncodingInfo =
       materializeEncodingFn(tensorType);
   if (failed(materializeEncodingInfo)) {
     return tensorType;
   }
   return tensor::PackOp::inferPackedType(
-             tensorType, materializeEncodingInfo->innerTileSizes,
+             getOriginalTypeWithEncoding(tensorType),
+             materializeEncodingInfo->innerTileSizes,
              materializeEncodingInfo->innerDimsPos,
              materializeEncodingInfo->outerDimsPerm)
       .cast<RankedTensorType>();
@@ -122,7 +141,6 @@
   }
   // Create `tensor.empty` operation for the result of the pack operation.
   Location loc = encodingOp.getLoc();
-  SmallVector<OpFoldResult> sourceDims = getDims(rewriter, loc, source);
   FailureOr<SmallVector<OpFoldResult>> innerTileSizesOfr =
       getInnerTileSizesOfr(rewriter, loc, resultType, *materializeEncodingInfo,
                            materializeEncodingValueFn);
@@ -133,13 +151,14 @@
   auto encoding = getEncodingAttr(resultType);
   if (!encoding)
     return failure();
+  std::optional<Value> paddingValue = getPaddingValue(source);
+  SmallVector<OpFoldResult> sourceDims = getDims(rewriter, loc, source);
   SmallVector<OpFoldResult> resultDims = tensor::PackOp::getResultShape(
       rewriter, loc, sourceDims, *innerTileSizesOfr,
       materializeEncodingInfo->innerDimsPos,
       materializeEncodingInfo->outerDimsPerm);
   auto emptyOp = rewriter.create<tensor::EmptyOp>(loc, resultDims,
                                                   resultType.getElementType());
-  std::optional<Value> paddingValue = getPaddingValue(source);
   return rewriter.create<tensor::PackOp>(
       loc, source, emptyOp, materializeEncodingInfo->innerDimsPos,
       *innerTileSizesOfr, paddingValue, materializeEncodingInfo->outerDimsPerm);
@@ -176,6 +195,31 @@
       *innerTileSizesOfr, materializeEncodingInfo->outerDimsPerm);
 }
 
+static FailureOr<SmallVector<Value>> lowerUpperBoundTileSizeOpToConstants(
+    RewriterBase &rewriter, UpperBoundTileSizeOp upperBoundTileSizeOp,
+    MaterializeEncodingFn materializeEncodingFn) {
+  Location loc = upperBoundTileSizeOp.getLoc();
+  RankedTensorType tensorType = upperBoundTileSizeOp.getTensorType();
+  FailureOr<MaterializeEncodingInfo> materializeEncodingInfo =
+      materializeEncodingFn(tensorType);
+  if (failed(materializeEncodingInfo)) {
+    return rewriter.notifyMatchFailure(upperBoundTileSizeOp,
+                                       "unhandled source encoding");
+  }
+  ArrayRef<int64_t> innerTileSizes = materializeEncodingInfo->innerTileSizes;
+  ArrayRef<int64_t> innerDimsPos = materializeEncodingInfo->innerDimsPos;
+  SmallVector<Value> results(tensorType.getRank());
+  for (unsigned i = 0; i < innerTileSizes.size(); ++i) {
+    int64_t tileSize = innerTileSizes[i];
+    if (ShapedType::isDynamic(tileSize)) {
+      tileSize = 16;
+    }
+    results[innerDimsPos[i]] =
+        rewriter.create<arith::ConstantIndexOp>(loc, tileSize);
+  }
+  return results;
+}
+
 /// Utility method to convert from `linalg.matmul` with
 /// - lhs encoding with role=LHS
 /// - rhs encoding with role=RHS
@@ -235,8 +279,8 @@
                     ValueRange convertedOperands,
                     MaterializeEncodingFn materializeEncodingFn,
                     MaterializeEncodingValueFn materializeEncodingValueFn) {
-  auto result = emptyOp.getResult();
-  auto resultType = result.getType().cast<RankedTensorType>();
+  auto resultType = getOriginalTypeWithEncoding(
+      emptyOp->getResultTypes()[0].cast<RankedTensorType>());
   FailureOr<MaterializeEncodingInfo> materializeEncodingInfo =
       materializeEncodingFn(resultType);
   if (failed(materializeEncodingInfo)) {
@@ -250,12 +294,15 @@
     return rewriter.notifyMatchFailure(
         emptyOp, "failed to generate runtime tile size query");
   }
-  SmallVector<OpFoldResult> newShape = PackOp::getResultShape(
-      rewriter, loc, emptyOp.getMixedSizes(), *innerTileSizesOfr,
-      materializeEncodingInfo->innerDimsPos,
-      materializeEncodingInfo->outerDimsPerm);
+  SmallVector<OpFoldResult> sourceDims = getMixedValues(
+      resultType.getShape(), emptyOp.getDynamicSizes(), rewriter);
+  SmallVector<OpFoldResult> newShape =
+      PackOp::getResultShape(rewriter, loc, sourceDims, *innerTileSizesOfr,
+                             materializeEncodingInfo->innerDimsPos,
+                             materializeEncodingInfo->outerDimsPerm);
   Operation *newEmptyOp = rewriter.create<tensor::EmptyOp>(
       loc, newShape, resultType.getElementType());
+
   return newEmptyOp;
 }
 
@@ -278,8 +325,6 @@
     MaterializeEncodingFn materializeEncodingFn =
         static_cast<MaterializeEncodingTypeConverter *>(getTypeConverter())
             ->getMaterializeEncodingFn();
-    // Pack op needs a padding value. Maybe that is an overkill. For now, just
-    // use zero.
     auto packOp = lowerSetEncodingOpToPackOp(
         rewriter, encodingOp, adaptor.getSource(), materializeEncodingFn,
         this->materializeEncodingValueFn);
@@ -315,6 +360,30 @@
   }
 };
 
+/// Convert `upper_bound_tile_size` op to `constant` op.
+struct UpperBoundTileSizeToConstantOpConversion
+    : public OpRewritePattern<UpperBoundTileSizeOp> {
+  UpperBoundTileSizeToConstantOpConversion(
+      MLIRContext *context, MaterializeEncodingFn materializeEncodingFn)
+      : OpRewritePattern<UpperBoundTileSizeOp>(context),
+        materializeEncodingFn(materializeEncodingFn) {}
+
+  LogicalResult matchAndRewrite(UpperBoundTileSizeOp upperBoundTileSizeOp,
+                                PatternRewriter &rewriter) const override {
+
+    auto constants = lowerUpperBoundTileSizeOpToConstants(
+        rewriter, upperBoundTileSizeOp, materializeEncodingFn);
+    if (failed(constants)) {
+      return rewriter.notifyMatchFailure(upperBoundTileSizeOp,
+                                         "failed to convert to constant op");
+    }
+    rewriter.replaceOp(upperBoundTileSizeOp, *constants);
+    return success();
+  }
+
+  MaterializeEncodingFn materializeEncodingFn;
+};
+
 /// Generic pattern to convert operaiton that is in Destination Passing Style.
 template <typename OpTy>
 struct MaterializeDPSOperation : public OpMaterializeEncodingPattern<OpTy> {
@@ -450,6 +519,12 @@
   ::mlir::memref::populateResolveRankedShapedTypeResultDimsPatterns(patterns);
 }
 
+void populateMaterializeUpperBoundTileSizePatterns(
+    RewritePatternSet &patterns, MaterializeEncodingFn materializeEncodingFn) {
+  patterns.insert<UpperBoundTileSizeToConstantOpConversion>(
+      patterns.getContext(), materializeEncodingFn);
+}
+
 std::unique_ptr<OperationPass<func::FuncOp>> createMaterializeEncodingPass() {
   return std::make_unique<MaterializeEncodingPass>();
 }
diff --git a/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/SplitReduction.cpp b/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/SplitReduction.cpp
index e8a3807..5680b48 100644
--- a/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/SplitReduction.cpp
+++ b/llvm-external-projects/iree-dialects/lib/Dialect/LinalgExt/Passes/SplitReduction.cpp
@@ -72,8 +72,8 @@
                                  int64_t splitReductionDepth) {
   // Determine if we should split the reduction. Requires aligned static shapes
   // and no input indicies.
-  auto valuesOrigType = topkOp.getInputType();
-  if (valuesOrigType.isDynamicDim(kDimOrig)) {
+  auto valuesOriginalType = topkOp.getInputType();
+  if (valuesOriginalType.isDynamicDim(kDimOrig)) {
     return rewriter.notifyMatchFailure(topkOp,
                                        "cannot split dynamic dimension");
   }
@@ -84,7 +84,7 @@
   if (splitReductionRatio <= 1) {
     return rewriter.notifyMatchFailure(topkOp, "reduction ratio <= 1");
   }
-  if (valuesOrigType.getDimSize(kDimOrig) % splitReductionRatio != 0) {
+  if (valuesOriginalType.getDimSize(kDimOrig) % splitReductionRatio != 0) {
     return rewriter.notifyMatchFailure(
         topkOp,
         "reduction dimension must be perfectly aligned to (divisible by) the "
@@ -102,13 +102,13 @@
                     int64_t splitReductionRatio, int64_t splitDimParallel,
                     int64_t kDimParallel, int64_t kSize) {
   Value valuesOrig = topkOp.values();
-  auto valuesOrigType = valuesOrig.getType().cast<ShapedType>();
-  Type valueElementType = valuesOrigType.getElementType();
+  auto valuesOriginalType = valuesOrig.getType().cast<ShapedType>();
+  Type valueElementType = valuesOriginalType.getElementType();
   Type indicesElementType =
       topkOp.getResultTypes()[1].cast<ShapedType>().getElementType();
 
   SmallVector<int64_t> expandedShape = getExpandedShape(
-      valuesOrigType.getShape(), splitReductionRatio, splitDimParallel);
+      valuesOriginalType.getShape(), splitReductionRatio, splitDimParallel);
   auto valuesExpandedType =
       RankedTensorType::get(expandedShape, valueElementType);
 
@@ -236,14 +236,14 @@
                             int64_t splitReductionRatio, int64_t kDimOrig,
                             int64_t kSize) {
   Value valuesOrig = topkOp.values();
-  auto valuesOrigType = valuesOrig.getType().cast<ShapedType>();
-  Type valueElementType = valuesOrigType.getElementType();
+  auto valuesOriginalType = valuesOrig.getType().cast<ShapedType>();
+  Type valueElementType = valuesOriginalType.getElementType();
   Type indicesElementType =
       topkOp.getResultTypes()[1].cast<ShapedType>().getElementType();
 
   // Define the collapsed input shapes
   SmallVector<int64_t> collapsedShape = getCollapsedShape(
-      valuesOrigType.getShape(), splitReductionRatio, kSize, kDimOrig);
+      valuesOriginalType.getShape(), splitReductionRatio, kSize, kDimOrig);
   auto valuesCollapsedType =
       RankedTensorType::get(collapsedShape, valueElementType);
   auto indicesCollapsedType =
diff --git a/tests/e2e/matmul/generate_e2e_matmul_tests.py b/tests/e2e/matmul/generate_e2e_matmul_tests.py
index 53dbbb4..d859fe2 100644
--- a/tests/e2e/matmul/generate_e2e_matmul_tests.py
+++ b/tests/e2e/matmul/generate_e2e_matmul_tests.py
@@ -113,6 +113,11 @@
             # square matrices. Start by the simplest case of 1x1x1.
             TestShape(m=1, k=1, n=1, accumulate=True),
             TestShape(m=1, k=1, n=1, accumulate=False),
+            # Test some small powers of two, that exercise in particular the
+            # adjustment of data-tiling tile sizes to narrow cases.
+            TestShape(m=2, k=2, n=2, accumulate=True),
+            TestShape(m=4, k=4, n=4, accumulate=True),
+            TestShape(m=8, k=8, n=8, accumulate=True),
             # test 9x9x9 because as many kernel M0/K0/N0 dims are equal to 8,
             # this will often be the smallest value that exercises something above
             # the kernel's size.