[NFC][Flow] Remove use of fusion preprocessing when it isnt a preprocessing (#17899)

The Fusion preprocessing pass was used in multiple places, which is
not the intent of the pass. Remove the subsequent usage.  The only
reason for this double usage was for the pattern that moved reduction
dimensions to the innermost. Consolidate that pattern with the pattern
in `InterchangeTransposeGenericPass` (whose name is very convoluted
and does not represent what it actually does).

This commit also includes the following changes:
- Rename `InterchangeTransposeGenericPass` to `TransposeGenericOpsPass`.
- Reoder the passes in `Passes.td` to be alphabetical within each of the
following
  portions
  - Dispatch region preprocessing passes
  - Dispatch region formation passes
  - General flow passes.

---------

Signed-off-by: MaheshRavishankar <mahesh.ravishankar@gmail.com>
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/BUILD.bazel b/compiler/src/iree/compiler/Dialect/Flow/Transforms/BUILD.bazel
index 88b43b5..5d56e7d 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/BUILD.bazel
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/BUILD.bazel
@@ -58,7 +58,6 @@
         "InjectDispatchTracing.cpp",
         "InjectTensorTracing.cpp",
         "InsertDispatchDebugTargets.cpp",
-        "InterchangeTransposeGenericOps.cpp",
         "MaterializeDefaultWorkgroupCountRegion.cpp",
         "OutlineConstants.cpp",
         "OutlineDispatchExterns.cpp",
@@ -69,6 +68,7 @@
         "SplitReduction.cpp",
         "TensorPadToTensorInsertSlice.cpp",
         "TopLevelSCFToCFG.cpp",
+        "TransposeGenericOps.cpp",
         "VerifyInputLegality.cpp",
     ],
     hdrs = [
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/CMakeLists.txt b/compiler/src/iree/compiler/Dialect/Flow/Transforms/CMakeLists.txt
index 95a30c6..2eeaa89 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/CMakeLists.txt
@@ -58,7 +58,6 @@
     "InjectDispatchTracing.cpp"
     "InjectTensorTracing.cpp"
     "InsertDispatchDebugTargets.cpp"
-    "InterchangeTransposeGenericOps.cpp"
     "MaterializeDefaultWorkgroupCountRegion.cpp"
     "OutlineConstants.cpp"
     "OutlineDispatchExterns.cpp"
@@ -69,6 +68,7 @@
     "SplitReduction.cpp"
     "TensorPadToTensorInsertSlice.cpp"
     "TopLevelSCFToCFG.cpp"
+    "TransposeGenericOps.cpp"
     "VerifyInputLegality.cpp"
   DEPS
     ::PassesIncGen
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/ConvertRegionToWorkgroups.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/ConvertRegionToWorkgroups.cpp
index c1bd272..fcd5bf3 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/ConvertRegionToWorkgroups.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/ConvertRegionToWorkgroups.cpp
@@ -18,9 +18,6 @@
 
 namespace mlir::iree_compiler::IREE::Flow {
 
-#define GEN_PASS_DEF_CONVERTREGIONTOWORKGROUPSPASS
-#include "iree/compiler/Dialect/Flow/Transforms/Passes.h.inc"
-
 namespace {
 
 /// Compute the dynamic dims of the given value and add them to the vector.
@@ -256,26 +253,4 @@
   return workgroupsOp;
 }
 
-namespace {
-struct ConvertRegionToWorkgroupsPass
-    : public IREE::Flow::impl::ConvertRegionToWorkgroupsPassBase<
-          ConvertRegionToWorkgroupsPass> {
-  void runOnOperation() override {
-    SmallVector<IREE::Flow::DispatchRegionOp> ops;
-    getOperation()->walk(
-        [&](IREE::Flow::DispatchRegionOp op) { ops.push_back(op); });
-
-    IRRewriter rewriter(getOperation()->getContext());
-    for (IREE::Flow::DispatchRegionOp regionOp : ops) {
-      if (failed(rewriteFlowDispatchRegionToFlowDispatchWorkgroups(regionOp,
-                                                                   rewriter))) {
-        signalPassFailure();
-        return;
-      }
-    }
-  }
-};
-
-} // namespace
-
 } // namespace mlir::iree_compiler::IREE::Flow
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/FusionPreprocessing.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/FusionPreprocessing.cpp
index 4aa3fc1..49d4859 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/FusionPreprocessing.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/FusionPreprocessing.cpp
@@ -37,39 +37,6 @@
 namespace {
 
 //===----------------------------------------------------------------------===//
-// GenericOpInterchangePattern
-//===----------------------------------------------------------------------===//
-
-struct GenericOpInterchangePattern
-    : public OpRewritePattern<linalg::GenericOp> {
-  using OpRewritePattern<linalg::GenericOp>::OpRewritePattern;
-  LogicalResult matchAndRewrite(linalg::GenericOp genericOp,
-                                PatternRewriter &rewriter) const override {
-    SmallVector<unsigned> interchange;
-    bool needInterchange = false;
-    unsigned numParallelLoop = genericOp.getNumParallelLoops();
-    if (numParallelLoop == 0)
-      return failure();
-    for (auto iter : llvm::enumerate(genericOp.getIteratorTypesArray())) {
-      if (linalg::isParallelIterator(iter.value())) {
-        interchange.push_back(iter.index());
-        if (iter.index() >= numParallelLoop)
-          needInterchange = true;
-      }
-    }
-    // If all the parallel loops are outter loops skip the pattern.
-    if (!needInterchange)
-      return failure();
-    for (auto iter : llvm::enumerate(genericOp.getIteratorTypesArray())) {
-      if (linalg::isReductionIterator(iter.value())) {
-        interchange.push_back(iter.index());
-      }
-    }
-    return interchangeGenericOp(rewriter, genericOp, interchange);
-  }
-};
-
-//===----------------------------------------------------------------------===//
 // ElementwiseOpInterchangePattern
 //===----------------------------------------------------------------------===//
 
@@ -235,8 +202,7 @@
   void runOnOperation() override {
     RewritePatternSet patterns(&getContext());
     patterns.add<ElementwiseOpInterchangePattern,
-                 FoldSuccessiveTensorInsertSliceOps,
-                 GenericOpInterchangePattern, GatherFusionPattern>(
+                 FoldSuccessiveTensorInsertSliceOps, GatherFusionPattern>(
         &getContext());
 
     // Fold away `tensor.dim` operations that can be resolved in terms of its
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.cpp
index 9ab790e..1693f49 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.cpp
@@ -185,7 +185,35 @@
       //    producer-consumer fusion.
       .addPass(IREE::Flow::createSinkReshapesPass)
       .addPass(IREE::Flow::createCanonicalizerPass)
-      .addPass(mlir::createCSEPass);
+      .addPass(mlir::createCSEPass)
+
+      // 5. After all the reshape propagations, fuse elementwise operations
+      //    even if the producer has multiple uses.
+      .addPass(IREE::Flow::createFuseMultiUseElementwiseProducerPass)
+
+      // 6. Some more "post elementwise fusion passes".
+      //    a. Detensorize.
+      //       TODO: This is probably not in the right place.
+      .addPredicatedPass(clDetensoring,
+                         [&]() { return mlir::createLinalgDetensorizePass(); })
+      .addPass(IREE::Flow::createCanonicalizerPass)
+      .addPass(mlir::createCSEPass)
+
+      //    b. For ops with multiple reduction dimensions, collapse the
+      //       reduction dimension.
+      //       TODO: This pass is only needed till all backends can handle
+      //       multiple reduction dimensions.
+      .addPredicatedPass(clCollapseReductionDims,
+                         IREE::Flow::createCollapseReductionDimensionsPass)
+
+      //     c. Split reduction operations into parallel and reduction, i.e
+      //        .
+      .addPass(IREE::Flow::createSplitReductionPass)
+
+      //     d. Transpose generic ops to
+      //        - help with dispatch region formation.
+      //        - move reduction iterators to be innermost.
+      .addPass(IREE::Flow::createTransposeGenericOpsPass);
 }
 
 // Pipeline to first create `flow.dispatch.region` ops and then lower to
@@ -207,7 +235,7 @@
       // Create dispatches for scalar operations as roots
       .addPass(IREE::Flow::createFormScalarDispatchesPass)
       // Create `flow.dispatch.region` centered around a root and fuse with
-      // producers
+      // producers and consumers.
       .addPass([&]() {
         return IREE::Flow::createFormDispatchRegionsPass(
             FormDispatchRegionsPassOptions{
@@ -256,25 +284,6 @@
       .addPass(mlir::createCSEPass);
 
   addDispatchRegionCreationPreprocessingPasses(passManager);
-
-  FunctionLikeNest(passManager)
-      .addPass(IREE::Flow::createFuseMultiUseElementwiseProducerPass)
-      .addPredicatedPass(clDetensoring,
-                         [&]() { return mlir::createLinalgDetensorizePass(); })
-      .addPass(IREE::Flow::createCanonicalizerPass)
-      .addPass(mlir::createCSEPass)
-      .addPredicatedPass(clCollapseReductionDims,
-                         IREE::Flow::createCollapseReductionDimensionsPass)
-      // Split reduction operations into parallel and reduction.
-      .addPass(IREE::Flow::createSplitReductionPass)
-      // SplitReductionPass may create reduction dimension that are not the last
-      // dimension.
-      .addPass(IREE::Flow::createFusionPreprocessingPass)
-      // Normalize the input indexing map to make the input indexing map
-      // identity. This helps fusing named linalg op with a generic op with
-      // transpose.
-      .addPass(IREE::Flow::createInterchangeTransposeGenericOpsPass);
-
   addDispatchRegionCreationPasses(passManager);
 }
 
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td b/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td
index b95bc79..ebfc5c0 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/Passes.td
@@ -9,10 +9,18 @@
 
 include "mlir/Pass/PassBase.td"
 
-def AnnotateDispatchesPass :
-    Pass<"iree-flow-annotate-dispatches", "mlir::ModuleOp"> {
-  let summary = "Annotates executable dispatches based on their contents.";
-}
+// File organization:
+// Groups passes that are related under one banner //===....===//. For example
+// the dispatch region creation preprocessing passes and dispatch region
+// formation passes are a couple of such groups. For any new pass add it to the
+// relevant group and keep them alphabetical within a group.
+
+//===---------------------------------------------------------------------===//
+// Dispatch region creation preprocessing passes :
+// Passes that transform the program before forming dispatches, like
+// - Elementwise operation fusion
+// - Reshape propagation passes
+//===---------------------------------------------------------------------===//
 
 def BubbleUpExpandShapesPass :
     Pass<"iree-flow-bubble-up-expand-shapes"> {
@@ -22,6 +30,254 @@
   ];
 }
 
+def CollapseReductionDimensionsPass :
+    Pass<"iree-flow-collapse-reduction-dimensions", ""> {
+  let summary = "Collapse reduction dimensions when possible.";
+  let dependentDialects = [
+    "mlir::linalg::LinalgDialect",
+  ];
+}
+
+def ElementwiseOpFusionPass :
+    Pass<"iree-flow-elementwise-op-fusion", ""> {
+  let summary = "Fuse elementwise operations.";
+  let options = [
+    Option<"fuseMultiReduction", "fuse-multi-reduction", "bool",
+           /*default=*/"true", "Fuse ops that have multiple reduction iterators">
+  ];
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+  ];
+}
+
+def FoldUnitExtentDimsPass :
+    Pass<"iree-flow-fold-unit-extent-dims", "mlir::ModuleOp"> {
+  let summary = "Fold unit extent dimension of operations.";
+  let description = [{
+    Imports upstream patterns to fold unit extent dims but with IREE control.
+  }];
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::arith::ArithDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::tensor::TensorDialect",
+  ];
+}
+
+def FuseMultiUseElementwiseProducerPass :
+    InterfacePass<"iree-flow-fuse-multi-use-elementwise-producer",
+                   "mlir::FunctionOpInterface"> {
+  let summary = "Fuse elementwise linalg operations on tensors when producers have multiple uses.";
+  let options = [
+    Option<"numIterations", "num-iterations", "unsigned",
+           /*default=*/"2", "Number of iterations to fuse multiuse ops">
+  ];
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::arith::ArithDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::math::MathDialect",
+  ];
+}
+
+def FusionPreprocessingPass :
+    Pass<"iree-flow-fusion-preprocessing", ""> {
+  let summary = "Run useful preprocessing patterns that help with fusion.";
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+  ];
+}
+
+def SinkReshapesPass :
+    Pass<"iree-flow-sink-reshapes", ""> {
+  let summary = "Sink reshapes to allow for compute op -> consumer fusion.";
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::arith::ArithDialect",
+  ];
+}
+
+def SplitReductionPass :
+    Pass<"iree-flow-split-reduction-ops", ""> {
+  let summary = "Split reduction dimension to increase parallelism.";
+  let dependentDialects = [
+    "mlir::linalg::LinalgDialect",
+  ];
+}
+
+def TensorPadToTensorInsertSlicePass :
+    Pass<"iree-flow-tensor-pad-to-tensor-insert-slice", ""> {
+  let summary = "Convert tensor.pad into linalg.fill + tensor.insert_slice.";
+  let options = [
+    Option<"skipSingleLinalgOpUses", "skip-one-linalg-use-case", "bool",
+           /*default=*/"false",
+           "Skip the op that has only one use which is used"
+           "by a Linalg op">,
+  ];
+  let dependentDialects = [
+    "mlir::arith::ArithDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::math::MathDialect",
+    "mlir::memref::MemRefDialect",
+  ];
+}
+
+def TransposeGenericOpsPass :
+    Pass<"iree-flow-transpose-generic-ops", ""> {
+  let summary = "Transpose generic op loops.";
+  let dependentDialects = [
+    "mlir::linalg::LinalgDialect",
+  ];
+}
+
+//===---------------------------------------------------------------------===//
+// Dispatch region creation passes.
+//===---------------------------------------------------------------------===//
+
+def CloneProducersIntoDispatchRegionsPass :
+    InterfacePass<"iree-flow-clone-producers-into-dispatch-regions", "mlir::FunctionOpInterface"> {
+  let summary = "Clone producers into dispatch regions to be isolated above.";
+  let description = [{
+    Pass to clone into dispatch regions producers of values used in the dispatch
+    regions but defined in the above. This prepares the dispatch regions for
+    converting to dispatch workgroups with explicit captures.
+  }];
+}
+
+def CollapseDimensionsPass :
+    InterfacePass<"iree-flow-collapse-dimensions", "mlir::FunctionOpInterface"> {
+  let summary = "Collapse dimensions of Linalg Ops on tensor ops.";
+  let description = [{
+    Collapse dimensions of Linalg Ops on tensor ops inside dispatch.region ops
+    and hoist the reshaping operations out of the dispatch.
+  }];
+}
+
+def ConvertDispatchRegionsToWorkgroupsPass :
+    InterfacePass<"iree-flow-convert-dispatch-regions-to-workgroups", "mlir::FunctionOpInterface"> {
+  let summary = "Convert dispatch regions to dispatch workgroups.";
+  let description = [{
+    Pass to convert dispatch regions to dispatch workgroups. This pass is
+    intended to be used after dispatch regions have been formed.
+  }];
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::scf::SCFDialect",
+    "mlir::tensor::TensorDialect",
+    "IREE::Flow::FlowDialect",
+  ];
+}
+
+def ConvertTensorToFlowPass :
+    InterfacePass<"iree-flow-convert-tensor-to-flow", "mlir::FunctionOpInterface"> {
+    let summary = "Convert tensor operations to flow";
+    let description = [{
+      Pass to convert tensor operations to flow.tensor.* operations.
+    }];
+    let dependentDialects = [
+      "mlir::affine::AffineDialect",
+      "mlir::arith::ArithDialect",
+      "mlir::linalg::LinalgDialect",
+      "mlir::tensor::TensorDialect",
+      "IREE::Flow::FlowDialect",
+    ];
+}
+
+def DispatchWithTransformDialectPass : Pass<"iree-flow-dispatch-with-transform-dialect"> {
+  let summary = "Dispatch Linalg operations on tensors by using the transform dialect interpreter.";
+  let description = [{
+    Pass to perform dispatch of Linalg on tensor ops by using the transform
+    dialect. Dispatch regions are created as specified by the transform module
+    that is parsed from `transformSpecPath`.
+
+    TODO: Drop this pass in favor of the one upstream. The one upstream requires
+    separate loading of the module and thus isn't suited for single-use
+    transform scripts.
+  }];
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::arith::ArithDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::pdl::PDLDialect",
+    "mlir::pdl_interp::PDLInterpDialect",
+    "mlir::scf::SCFDialect",
+    "mlir::tensor::TensorDialect",
+    "mlir::transform::TransformDialect",
+    "IREE::Flow::FlowDialect",
+    "IREE::LinalgExt::IREELinalgExtDialect",
+  ];
+  let options = [
+    Option<"disableExpensiveChecks", "disable-expensive-checks", "bool",
+           "false",
+           "Disable expensive checks in the interpreter for a faster run.">,
+    Option<"transformSpecPath", "transform-spec-path", "std::string",
+           /*default=*/"", "File path to the transform spec to use.">,
+  ];
+}
+
+def FormDispatchRegionsPass :
+    InterfacePass<"iree-flow-form-dispatch-regions", "mlir::FunctionOpInterface"> {
+  let summary = "Form Dispatch Region Ops from Linalg operations on tensors to form dispatch.regions.";
+  let options = [
+    Option<"aggressiveFusion", "aggressive-fusion", "bool",
+           /*default=*/"false", "Aggressive mode enabling fusions not ready for all backends">,
+    Option<"fusePadWithConsumers", "fuse-pad-with-consumers", "bool",
+           /*default=*/"false", "Enable fusing pad with consumer">,
+    Option<"fusePadWithProducers", "fuse-pad-with-producers", "bool",
+           /*default=*/"false", "Enable fusion of pad with producers">
+  ];
+  let description = [{
+    Pass to form dispatch.region ops from Linalg on tensor ops. A dispatch region
+    is created for each tiled loop nest. This pass only moves the root compute op
+    into the dispatch region, allowing producers to be outside.
+  }];
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::scf::SCFDialect",
+    "mlir::tensor::TensorDialect",
+    "IREE::Flow::FlowDialect",
+    "IREE::LinalgExt::IREELinalgExtDialect",
+  ];
+}
+
+def FormScalarDispatchesPass :
+    InterfacePass<"iree-flow-form-scalar-dispatches", "mlir::FunctionOpInterface"> {
+  let summary = "Form Dispatch Regions for scalar computations.";
+  let dependentDialects = [
+    "mlir::affine::AffineDialect",
+    "mlir::linalg::LinalgDialect",
+    "mlir::tensor::TensorDialect",
+    "IREE::Flow::FlowDialect",
+  ];
+}
+
+def MaterializeDefaultWorkgroupCountRegionPass:
+    InterfacePass<"iree-flow-materialize-default-workgroup-count-region",
+                  "mlir::FunctionOpInterface"> {
+      let summary = "Canonicalize dispatch workgroups ops.";
+      let description = [{
+        Apply dispatch workgroups canonicalization patterns.
+      }];
+      let dependentDialects = [
+        "mlir::affine::AffineDialect",
+        "mlir::arith::ArithDialect",
+        "mlir::linalg::LinalgDialect",
+        "mlir::scf::SCFDialect",
+        "IREE::Flow::FlowDialect",
+      ];
+}
+
+//===---------------------------------------------------------------------===//
+// General Flow passes
+//===---------------------------------------------------------------------===//
+
+def AnnotateDispatchesPass :
+    Pass<"iree-flow-annotate-dispatches", "mlir::ModuleOp"> {
+  let summary = "Annotates executable dispatches based on their contents.";
+}
+
 def CanonicalizerPass :
     Pass<"iree-flow-canonicalize", ""> {
   let summary = "Flow specific canonicalization pass";
@@ -44,24 +300,6 @@
   let summary = "Cleans up any remaining tensor shape metadata after lowering.";
 }
 
-def CloneProducersIntoDispatchRegionsPass :
-    InterfacePass<"iree-flow-clone-producers-into-dispatch-regions", "mlir::FunctionOpInterface"> {
-  let summary = "Clone producers into dispatch regions to be isolated above.";
-  let description = [{
-    Pass to clone into dispatch regions producers of values used in the dispatch
-    regions but defined in the above. This prepares the dispatch regions for
-    converting to dispatch workgroups with explicit captures.
-  }];
-}
-
-def CollapseReductionDimensionsPass :
-    Pass<"iree-flow-collapse-reduction-dimensions", ""> {
-  let summary = "Collapse reduction dimensions when possible.";
-  let dependentDialects = [
-    "mlir::linalg::LinalgDialect",
-  ];
-}
-
 def ConvertMeshToFlowPass :
     Pass<"iree-convert-mesh-to-flow", "mlir::ModuleOp"> {
   let summary = "Convert Mesh dialect operations to flow.";
@@ -109,15 +347,6 @@
   ];
 }
 
-def ConvertRegionToWorkgroupsPass :
-    Pass<"iree-flow-convert-region-to-workgroups", ""> {
-  let summary = "Convert dispatch.region ops to dispatch.workgroups ops.";
-  let dependentDialects = [
-    "mlir::tensor::TensorDialect",
-    "IREE::Flow::FlowDialect",
-  ];
-}
-
 def ConvertToFlowPass :
     Pass<"iree-flow-convert-to-flow", ""> {
   let summary = "Convert operations to flow. Currently just a test pass.";
@@ -140,144 +369,6 @@
   let summary = "Deduplicates executables that are identical.";
 }
 
-def FoldUnitExtentDimsPass :
-    Pass<"iree-flow-fold-unit-extent-dims", "mlir::ModuleOp"> {
-  let summary = "Fold unit extent dimension of operations.";
-  let description = [{
-    Imports upstream patterns to fold unit extent dims but with IREE control.
-  }];
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::arith::ArithDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::tensor::TensorDialect",
-  ];
-}
-
-def FormDispatchRegionsPass :
-    InterfacePass<"iree-flow-form-dispatch-regions", "mlir::FunctionOpInterface"> {
-  let summary = "Form Dispatch Region Ops from Linalg operations on tensors to form dispatch.regions.";
-  let options = [
-    Option<"aggressiveFusion", "aggressive-fusion", "bool",
-           /*default=*/"false", "Aggressive mode enabling fusions not ready for all backends">,
-    Option<"fusePadWithConsumers", "fuse-pad-with-consumers", "bool",
-           /*default=*/"false", "Enable fusing pad with consumer">,
-    Option<"fusePadWithProducers", "fuse-pad-with-producers", "bool",
-           /*default=*/"false", "Enable fusion of pad with producers">
-  ];
-  let description = [{
-    Pass to form dispatch.region ops from Linalg on tensor ops. A dispatch region
-    is created for each tiled loop nest. This pass only moves the root compute op
-    into the dispatch region, allowing producers to be outside.
-  }];
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::scf::SCFDialect",
-    "mlir::tensor::TensorDialect",
-    "IREE::Flow::FlowDialect",
-    "IREE::LinalgExt::IREELinalgExtDialect",
-  ];
-}
-
-def ConvertDispatchRegionsToWorkgroupsPass :
-    InterfacePass<"iree-flow-convert-dispatch-regions-to-workgroups", "mlir::FunctionOpInterface"> {
-  let summary = "Convert dispatch regions to dispatch workgroups.";
-  let description = [{
-    Pass to convert dispatch regions to dispatch workgroups. This pass is
-    intended to be used after dispatch regions have been formed.
-  }];
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::scf::SCFDialect",
-    "mlir::tensor::TensorDialect",
-    "IREE::Flow::FlowDialect",
-  ];
-}
-
-def ConvertTensorToFlowPass :
-    InterfacePass<"iree-flow-convert-tensor-to-flow", "mlir::FunctionOpInterface"> {
-    let summary = "Convert tensor operations to flow";
-    let description = [{
-      Pass to convert tensor operations to flow.tensor.* operations.
-    }];
-    let dependentDialects = [
-      "mlir::affine::AffineDialect",
-      "mlir::arith::ArithDialect",
-      "mlir::linalg::LinalgDialect",
-      "mlir::tensor::TensorDialect",
-      "IREE::Flow::FlowDialect",
-    ];
-}
-
-def MaterializeDefaultWorkgroupCountRegionPass:
-    InterfacePass<"iree-flow-materialize-default-workgroup-count-region", "mlir::FunctionOpInterface"> {
-      let summary = "Canonicalize dispatch workgroups ops.";
-      let description = [{
-        Apply dispatch workgroups canonicalization patterns.
-      }];
-      let dependentDialects = [
-        "mlir::affine::AffineDialect",
-        "mlir::arith::ArithDialect",
-        "mlir::linalg::LinalgDialect",
-        "mlir::scf::SCFDialect",
-        "IREE::Flow::FlowDialect",
-      ];
-}
-
-def FormScalarDispatchesPass :
-    InterfacePass<"iree-flow-form-scalar-dispatches", "mlir::FunctionOpInterface"> {
-  let summary = "Form Dispatch Regions for scalar computations.";
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::tensor::TensorDialect",
-    "IREE::Flow::FlowDialect",
-  ];
-}
-
-def CollapseDimensionsPass :
-    InterfacePass<"iree-flow-collapse-dimensions", "mlir::FunctionOpInterface"> {
-  let summary = "Collapse dimensions of Linalg Ops on tensor ops.";
-  let description = [{
-    Collapse dimensions of Linalg Ops on tensor ops inside dispatch.region ops
-    and hoist the reshaping operations out of the dispatch.
-  }];
-}
-
-def DispatchWithTransformDialectPass : Pass<"iree-flow-dispatch-with-transform-dialect"> {
-  let summary = "Dispatch Linalg operations on tensors by using the transform dialect interpreter.";
-  let description = [{
-    Pass to perform dispatch of Linalg on tensor ops by using the transform
-    dialect. Dispatch regions are created as specified by the transform module
-    that is parsed from `transformSpecPath`.
-
-    TODO: Drop this pass in favor of the one upstream. The one upstream requires
-    separate loading of the module and thus isn't suited for single-use
-    transform scripts.
-  }];
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::arith::ArithDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::pdl::PDLDialect",
-    "mlir::pdl_interp::PDLInterpDialect",
-    "mlir::scf::SCFDialect",
-    "mlir::tensor::TensorDialect",
-    "mlir::transform::TransformDialect",
-    "IREE::Flow::FlowDialect",
-    "IREE::LinalgExt::IREELinalgExtDialect",
-  ];
-  let options = [
-    Option<"disableExpensiveChecks", "disable-expensive-checks", "bool",
-           "false",
-           "Disable expensive checks in the interpreter for a faster run.">,
-    Option<"transformSpecPath", "transform-spec-path", "std::string",
-           /*default=*/"", "File path to the transform spec to use.">,
-  ];
-}
-
 def DumpDispatchGraphPass : Pass<"iree-flow-dump-dispatch-graph-pass"> {
   let summary = "Dump visualization of dispatches within the program.";
   let options = [
@@ -300,19 +391,6 @@
   ];
 }
 
-def ElementwiseOpFusionPass :
-    Pass<"iree-flow-elementwise-op-fusion", ""> {
-  let summary = "Fuse elementwise operations.";
-  let options = [
-    Option<"fuseMultiReduction", "fuse-multi-reduction", "bool",
-           /*default=*/"true", "Fuse ops that have multiple reduction iterators">
-  ];
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-  ];
-}
-
-
 def ExportBenchmarkFuncsPass :
     Pass<"iree-flow-export-benchmark-funcs-pass", "mlir::ModuleOp"> {
   let summary = "Exports benchmark functions.";
@@ -327,28 +405,6 @@
   ];
 }
 
-def FuseMultiUseElementwiseProducerPass :
-    InterfacePass<"iree-flow-fuse-multi-use-elementwise-producer", "mlir::FunctionOpInterface"> {
-  let summary = "Fuse elementwise linalg operations on tensors when producers have multiple uses.";
-  let options = [
-    Option<"numIterations", "num-iterations", "unsigned",
-           /*default=*/"2", "Number of iterations to fuse multiuse ops">
-  ];
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::arith::ArithDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::math::MathDialect",
-  ];
-}
-
-def FusionPreprocessingPass :
-    Pass<"iree-flow-fusion-preprocessing", ""> {
-  let summary = "Run useful preprocessing patterns that help with fusion.";
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-  ];
-}
 
 def InitializeEmptyTensorsPass :
     Pass<"iree-flow-initialize-empty-tensors", ""> {
@@ -418,14 +474,6 @@
   ];
 }
 
-def InterchangeTransposeGenericOpsPass :
-    Pass<"iree-flow-interchange-transpose-generic-ops", ""> {
-  let summary = "Interchange transpose generic op loops to make the input indeximg map indentity.";
-  let dependentDialects = [
-    "mlir::linalg::LinalgDialect",
-  ];
-}
-
 def OutlineConstantsPass :
     Pass<"iree-flow-outline-constants", "mlir::ModuleOp"> {
   let summary = "Outlines tensor constants into util.globals at the module level.";
@@ -458,40 +506,6 @@
   ];
 }
 
-def SinkReshapesPass :
-    Pass<"iree-flow-sink-reshapes", ""> {
-  let summary = "Sink reshapes to allow for compute op -> consumer fusion.";
-  let dependentDialects = [
-    "mlir::affine::AffineDialect",
-    "mlir::arith::ArithDialect",
-  ];
-}
-
-def SplitReductionPass :
-    Pass<"iree-flow-split-reduction-ops", ""> {
-  let summary = "Split reduction dimension to increase parallelism.";
-  let dependentDialects = [
-    "mlir::linalg::LinalgDialect",
-  ];
-}
-
-def TensorPadToTensorInsertSlicePass :
-    Pass<"iree-flow-tensor-pad-to-tensor-insert-slice", ""> {
-  let summary = "Convert tensor.pad into linalg.fill + tensor.insert_slice.";
-  let options = [
-    Option<"skipSingleLinalgOpUses", "skip-one-linalg-use-case", "bool",
-           /*default=*/"false",
-           "Skip the op that has only one use which is used"
-           "by a Linalg op">,
-  ];
-  let dependentDialects = [
-    "mlir::arith::ArithDialect",
-    "mlir::linalg::LinalgDialect",
-    "mlir::math::MathDialect",
-    "mlir::memref::MemRefDialect",
-  ];
-}
-
 def TopLevelSCFToCFGPass :
     InterfacePass<"iree-top-level-scf-to-cfg", "mlir::FunctionOpInterface"> {
   let summary = "Converts non-nested SCF constructs to CFG (not traversing into opaque operations).";
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/InterchangeTransposeGenericOps.cpp b/compiler/src/iree/compiler/Dialect/Flow/Transforms/TransposeGenericOps.cpp
similarity index 61%
rename from compiler/src/iree/compiler/Dialect/Flow/Transforms/InterchangeTransposeGenericOps.cpp
rename to compiler/src/iree/compiler/Dialect/Flow/Transforms/TransposeGenericOps.cpp
index 7a03bc8..ea05324 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/InterchangeTransposeGenericOps.cpp
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/TransposeGenericOps.cpp
@@ -18,11 +18,50 @@
 
 namespace mlir::iree_compiler::IREE::Flow {
 
-#define GEN_PASS_DEF_INTERCHANGETRANSPOSEGENERICOPSPASS
+#define GEN_PASS_DEF_TRANSPOSEGENERICOPSPASS
 #include "iree/compiler/Dialect/Flow/Transforms/Passes.h.inc"
 
 namespace {
 
+//===----------------------------------------------------------------------===//
+// MakeReductionInnermostPattern
+//===----------------------------------------------------------------------===//
+
+/// For generic ops that are reduction, make the reduction the innermost
+/// dimension.
+struct MakeReductionInnermostPattern
+    : public OpRewritePattern<linalg::GenericOp> {
+  using OpRewritePattern<linalg::GenericOp>::OpRewritePattern;
+  LogicalResult matchAndRewrite(linalg::GenericOp genericOp,
+                                PatternRewriter &rewriter) const override {
+    SmallVector<unsigned> interchange;
+    bool needInterchange = false;
+    unsigned numParallelLoop = genericOp.getNumParallelLoops();
+    if (numParallelLoop == 0)
+      return failure();
+    for (auto iter : llvm::enumerate(genericOp.getIteratorTypesArray())) {
+      if (linalg::isParallelIterator(iter.value())) {
+        interchange.push_back(iter.index());
+        if (iter.index() >= numParallelLoop)
+          needInterchange = true;
+      }
+    }
+    // If all the parallel loops are outter loops skip the pattern.
+    if (!needInterchange)
+      return failure();
+    for (auto iter : llvm::enumerate(genericOp.getIteratorTypesArray())) {
+      if (linalg::isReductionIterator(iter.value())) {
+        interchange.push_back(iter.index());
+      }
+    }
+    return interchangeGenericOp(rewriter, genericOp, interchange);
+  }
+};
+
+/// For elementwise ops that consumer values produced by named ops (or reduction
+/// ops), the dispatch region fusion logic requires the indexing maps to be
+/// identity (or projections that are not transposing as well). This pattern
+/// fixes up elementwise operations for which that is not the case.
 struct TransposeGenericOpPattern : public OpRewritePattern<linalg::GenericOp> {
   using OpRewritePattern<linalg::GenericOp>::OpRewritePattern;
   LogicalResult matchAndRewrite(linalg::GenericOp genericOp,
@@ -73,12 +112,13 @@
   }
 };
 
-struct InterchangeTransposeGenericOpsPass
-    : public IREE::Flow::impl::InterchangeTransposeGenericOpsPassBase<
-          InterchangeTransposeGenericOpsPass> {
+struct TransposeGenericOpsPass
+    : public IREE::Flow::impl::TransposeGenericOpsPassBase<
+          TransposeGenericOpsPass> {
   void runOnOperation() override {
     RewritePatternSet patterns(&getContext());
-    patterns.add<TransposeGenericOpPattern>(&getContext());
+    patterns.add<MakeReductionInnermostPattern, TransposeGenericOpPattern>(
+        &getContext());
     if (failed(applyPatternsAndFoldGreedily(getOperation(),
                                             std::move(patterns)))) {
       return signalPassFailure();
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/BUILD.bazel b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/BUILD.bazel
index 09e8511..dfd39ae 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/BUILD.bazel
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/BUILD.bazel
@@ -43,7 +43,6 @@
             "inject_dispatch_tracing.mlir",
             "inject_tensor_tracing.mlir",
             "insert_dispatch_debug_targets.mlir",
-            "interchange_transpose_generic_ops.mlir",
             "outline_constants.mlir",
             "outline_dispatch_externs.mlir",
             "outline_dispatch_regions.mlir",
@@ -55,6 +54,7 @@
             "tensor_pad_to_tensor_insert_slice.mlir",
             "top_level_scf_to_cfg.mlir",
             "transform_dispatch_region_formation.mlir",
+            "transpose_generic_ops.mlir",
             "verify_input_ir.mlir",
         ],
         include = ["*.mlir"],
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/CMakeLists.txt b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/CMakeLists.txt
index 9324301..71f8083 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/CMakeLists.txt
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/CMakeLists.txt
@@ -41,7 +41,6 @@
     "inject_dispatch_tracing.mlir"
     "inject_tensor_tracing.mlir"
     "insert_dispatch_debug_targets.mlir"
-    "interchange_transpose_generic_ops.mlir"
     "outline_constants.mlir"
     "outline_dispatch_externs.mlir"
     "outline_dispatch_regions.mlir"
@@ -53,6 +52,7 @@
     "tensor_pad_to_tensor_insert_slice.mlir"
     "top_level_scf_to_cfg.mlir"
     "transform_dispatch_region_formation.mlir"
+    "transpose_generic_ops.mlir"
     "verify_input_ir.mlir"
   TOOLS
     FileCheck
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/convert_region_to_workgroups.mlir b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/convert_region_to_workgroups.mlir
index 1f94c2e..3caa5b0 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/convert_region_to_workgroups.mlir
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/convert_region_to_workgroups.mlir
@@ -1,4 +1,4 @@
-// RUN: iree-opt %s --iree-flow-convert-region-to-workgroups --iree-flow-canonicalize -cse -split-input-file | FileCheck %s
+// RUN: iree-opt %s --pass-pipeline="builtin.module(util.func(iree-flow-convert-dispatch-regions-to-workgroups, iree-flow-canonicalize, cse))" -split-input-file | FileCheck %s
 
 // CHECK-LABEL: util.func public @foo(
 //       CHECK:   %[[argA:.*]]: tensor<?x?xf32>, %[[argB:.*]]: tensor<5x10xf32>, %[[argC:.*]]: tensor<10x11xf32>
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/dispatch_linalg_on_tensors_fusion_with_transpose.mlir b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/dispatch_linalg_on_tensors_fusion_with_transpose.mlir
index 4a9b2d8..a091b90 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/dispatch_linalg_on_tensors_fusion_with_transpose.mlir
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/dispatch_linalg_on_tensors_fusion_with_transpose.mlir
@@ -1,4 +1,4 @@
-// RUN: iree-opt --split-input-file --verify-diagnostics --pass-pipeline="builtin.module(util.func(iree-flow-interchange-transpose-generic-ops,iree-flow-form-dispatch-regions{aggressive-fusion=true}, iree-flow-convert-dispatch-regions-to-workgroups, iree-flow-canonicalize, cse))" --mlir-print-local-scope %s | FileCheck %s
+// RUN: iree-opt --split-input-file --verify-diagnostics --pass-pipeline="builtin.module(util.func(iree-flow-transpose-generic-ops,iree-flow-form-dispatch-regions{aggressive-fusion=true}, iree-flow-convert-dispatch-regions-to-workgroups, iree-flow-canonicalize, cse))" --mlir-print-local-scope %s | FileCheck %s
 
 util.func @fuse_conv(%arg0 : tensor<2x130x130x16xf32>, %arg1 : tensor<3x3x16x320xf32>) -> tensor<2x320x128x128xf32> {
   %empty = tensor.empty() : tensor<2x128x128x320xf32>
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/fusion_preprocessing.mlir b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/fusion_preprocessing.mlir
index 04713f0..b1865bc 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/fusion_preprocessing.mlir
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/fusion_preprocessing.mlir
@@ -1,29 +1,5 @@
 // RUN: iree-opt --iree-flow-fusion-preprocessing --split-input-file %s | FileCheck %s
 
-// CHECK: #[[MAP0:.+]] = affine_map<(d0, d1, d2, d3) -> (d0, d3, d2)>
-// CHECK: #[[MAP1:.+]] = affine_map<(d0, d1, d2, d3) -> (d3, d0, d1)>
-// CHECK: #[[MAP2:.+]] = affine_map<(d0, d1, d2, d3) -> (d2, d0, d1)>
-//      CHECK: util.func public @interchange
-//      CHECK:   linalg.generic {indexing_maps = [#[[MAP0]], #[[MAP1]], #[[MAP2]]]
-// CHECK-SAME:   iterator_types = ["parallel", "parallel", "parallel", "reduction"]}
-util.func public @interchange(%arg0: tensor<?x?x?xf32>, %arg1: tensor<?x?x?xf32>, %arg2: tensor<?x?x?xf32>) -> (tensor<?x?x?xf32>) {
-  %0 = linalg.generic {indexing_maps = [
-    affine_map<(d0, d1, d2, d3) -> (d1, d0, d3)>,
-    affine_map<(d0, d1, d2, d3) -> (d0, d1, d2)>,
-    affine_map<(d0, d1, d2, d3) -> (d3, d1, d2)>],
-    iterator_types = ["reduction", "parallel", "parallel", "parallel"]}
-  ins(%arg0, %arg1 : tensor<?x?x?xf32>, tensor<?x?x?xf32>)
-  outs(%arg2 : tensor<?x?x?xf32>) {
-  ^bb0(%arg3: f32, %arg4: f32, %arg5: f32):  // no predecessors
-    %m = arith.mulf %arg3, %arg4 : f32
-    %a = arith.addf %arg5, %m : f32
-    linalg.yield %a : f32
-  } -> tensor<?x?x?xf32>
-  util.return %0 : tensor<?x?x?xf32>
-}
-
-// -----
-
 util.func public @fold_insert_slices(%source : tensor<?x?xf32>,
     %dest0 : tensor<?x?xf32>, %dest1 : tensor<?x?xf32>, %val: f32,
     %o1 : index, %o2 : index, %o3 : index, %o4 : index,
diff --git a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/interchange_transpose_generic_ops.mlir b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/transpose_generic_ops.mlir
similarity index 71%
rename from compiler/src/iree/compiler/Dialect/Flow/Transforms/test/interchange_transpose_generic_ops.mlir
rename to compiler/src/iree/compiler/Dialect/Flow/Transforms/test/transpose_generic_ops.mlir
index 3a809ee..e857cdb 100644
--- a/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/interchange_transpose_generic_ops.mlir
+++ b/compiler/src/iree/compiler/Dialect/Flow/Transforms/test/transpose_generic_ops.mlir
@@ -1,4 +1,4 @@
-// RUN: iree-opt --split-input-file --verify-diagnostics --iree-flow-interchange-transpose-generic-ops --iree-flow-canonicalize -cse --mlir-print-local-scope %s | FileCheck %s
+// RUN: iree-opt --split-input-file --verify-diagnostics --iree-flow-transpose-generic-ops --iree-flow-canonicalize -cse --mlir-print-local-scope %s | FileCheck %s
 
 util.func @supported_conv(%arg0 : tensor<2x130x130x16xf16>, %arg1 : tensor<3x3x16x320xf16>) -> tensor<2x320x128x128xf16> {
   %empty = tensor.empty() : tensor<2x128x128x320xf32>
@@ -55,3 +55,27 @@
 //  CHECK-SAME:     indexing_maps = [affine_map<(d0, d1, d2, d3) -> (d0, d1, d2, d3)>,
 //  CHECK-SAME:     affine_map<(d0, d1, d2, d3) -> (d3, d0, d1, d2)>]
 //       CHECK:   return %[[RESULT]]
+
+//  -----
+
+//      CHECK: util.func public @interchange
+//      CHECK:   linalg.generic {indexing_maps = [
+// CHECK-SAME:       affine_map<(d0, d1, d2, d3) -> (d0, d3, d2)>,
+// CHECK-SAME:       affine_map<(d0, d1, d2, d3) -> (d3, d0, d1)>
+// CHECK-SAME:       affine_map<(d0, d1, d2, d3) -> (d2, d0, d1)>
+// CHECK-SAME:   iterator_types = ["parallel", "parallel", "parallel", "reduction"]}
+util.func public @interchange(%arg0: tensor<?x?x?xf32>, %arg1: tensor<?x?x?xf32>, %arg2: tensor<?x?x?xf32>) -> (tensor<?x?x?xf32>) {
+  %0 = linalg.generic {indexing_maps = [
+    affine_map<(d0, d1, d2, d3) -> (d1, d0, d3)>,
+    affine_map<(d0, d1, d2, d3) -> (d0, d1, d2)>,
+    affine_map<(d0, d1, d2, d3) -> (d3, d1, d2)>],
+    iterator_types = ["reduction", "parallel", "parallel", "parallel"]}
+  ins(%arg0, %arg1 : tensor<?x?x?xf32>, tensor<?x?x?xf32>)
+  outs(%arg2 : tensor<?x?x?xf32>) {
+  ^bb0(%arg3: f32, %arg4: f32, %arg5: f32):  // no predecessors
+    %m = arith.mulf %arg3, %arg4 : f32
+    %a = arith.addf %arg5, %m : f32
+    linalg.yield %a : f32
+  } -> tensor<?x?x?xf32>
+  util.return %0 : tensor<?x?x?xf32>
+}