Adding alignment analysis and propagation into codegen. (#7714)

This adds stream.alignment attributes to !stream.binding and index
operands in dispatch functions and stream.values attributes to integer
operands. The alignment on bindings indicates the base binding offset
alignment while the alignment on index operands indicates the alignment
of any offset into a binding they may represent. The values are all
potential values that may be passed in the operand when knowable.

The final piece required is to use this information in the codegen
backends, however the memref.assume_alignment op doesn't seem to do
anything today. This would in theory plumb the information through
but there are likely better ways to use this information:
```
auto alignment = baseBuffer.calculateAlignment();
b.create<memref::AssumeAlignmentOp>(op->getLoc(), baseBuffer,
                                    alignment.value());
```
diff --git a/iree/compiler/Codegen/Common/CleanupBufferAllocViewPass.cpp b/iree/compiler/Codegen/Common/CleanupBufferAllocViewPass.cpp
index 3cd3e88..0be9e12 100644
--- a/iree/compiler/Codegen/Common/CleanupBufferAllocViewPass.cpp
+++ b/iree/compiler/Codegen/Common/CleanupBufferAllocViewPass.cpp
@@ -85,7 +85,7 @@
     Value newSubspanOp = rewriter.create<IREE::HAL::InterfaceBindingSubspanOp>(
         subspanOp.getLoc(), newSubspanType, subspanOp.binding(),
         subspanOp.byte_offset(), subspanOp.byte_length(),
-        subspanOp.dynamic_dims());
+        subspanOp.dynamic_dims(), subspanOp.alignmentAttr());
 
     rewriter.replaceOpWithNewOp<IREE::Flow::DispatchTensorLoadOp>(
         reshapeOp, reshapeOp.getResultType(), newSubspanOp);
diff --git a/iree/compiler/Codegen/Common/FlattenMemRefSubspanPass.cpp b/iree/compiler/Codegen/Common/FlattenMemRefSubspanPass.cpp
index fb1fe5b..62bfb04 100644
--- a/iree/compiler/Codegen/Common/FlattenMemRefSubspanPass.cpp
+++ b/iree/compiler/Codegen/Common/FlattenMemRefSubspanPass.cpp
@@ -226,7 +226,7 @@
 
     rewriter.replaceOpWithNewOp<IREE::HAL::InterfaceBindingSubspanOp>(
         subspanOp, newType, subspanOp.binding(), subspanOp.byte_offset(),
-        subspanOp.byte_length(), dynamicDim);
+        subspanOp.byte_length(), dynamicDim, subspanOp.alignmentAttr());
     return success();
   }
 };
@@ -516,7 +516,8 @@
         rewriter.create<arith::ConstantIndexOp>(op.memref().getLoc(), 0);
     Value newSubspan = rewriter.create<IREE::HAL::InterfaceBindingSubspanOp>(
         op.memref().getLoc(), subspanOp.getType(), subspanOp.binding(), zero,
-        subspanOp.byte_length(), subspanOp.dynamic_dims());
+        subspanOp.byte_length(), subspanOp.dynamic_dims(),
+        subspanOp.alignmentAttr());
     rewriter.restoreInsertionPoint(ip);
 
     MLIRContext *context = rewriter.getContext();
diff --git a/iree/compiler/Codegen/Common/IREEComprehensiveBufferizePass.cpp b/iree/compiler/Codegen/Common/IREEComprehensiveBufferizePass.cpp
index 554039f..9fe6ee3 100644
--- a/iree/compiler/Codegen/Common/IREEComprehensiveBufferizePass.cpp
+++ b/iree/compiler/Codegen/Common/IREEComprehensiveBufferizePass.cpp
@@ -90,7 +90,7 @@
     auto baseBuffer = b.create<IREE::HAL::InterfaceBindingSubspanOp>(
         subspanOp->getLoc(), memRefType, subspanOp.binding(),
         subspanOp.byte_offset(), subspanOp.byte_length(),
-        subspanOp.dynamic_dims());
+        subspanOp.dynamic_dims(), subspanOp.alignmentAttr());
     state.mapValue(subspanOp, baseBuffer);
     state.aliasInfo.createAliasInfoEntry(subspanOp.result());
   }
diff --git a/iree/compiler/Codegen/Common/LinalgBufferizePass.cpp b/iree/compiler/Codegen/Common/LinalgBufferizePass.cpp
index 936dbf6..3238c69 100644
--- a/iree/compiler/Codegen/Common/LinalgBufferizePass.cpp
+++ b/iree/compiler/Codegen/Common/LinalgBufferizePass.cpp
@@ -921,7 +921,7 @@
     auto memRefType = getMemrefTypeForTensor(tensorType);
     auto baseBuffer = b.create<IREE::HAL::InterfaceBindingSubspanOp>(
         op->getLoc(), memRefType, op.binding(), op.byte_offset(),
-        op.byte_length(), op.dynamic_dims());
+        op.byte_length(), op.dynamic_dims(), op.alignmentAttr());
     bvm.map(op, baseBuffer);
   });
 
diff --git a/iree/compiler/Codegen/SPIRV/SPIRVVectorizeLoadStore.cpp b/iree/compiler/Codegen/SPIRV/SPIRVVectorizeLoadStore.cpp
index 4367451..b7b87ce 100644
--- a/iree/compiler/Codegen/SPIRV/SPIRVVectorizeLoadStore.cpp
+++ b/iree/compiler/Codegen/SPIRV/SPIRVVectorizeLoadStore.cpp
@@ -362,7 +362,8 @@
     }
     rewriter.replaceOpWithNewOp<IREE::HAL::InterfaceBindingSubspanOp>(
         bindingOp, *vecMemRef, bindingOp.binding(), bindingOp.byte_offset(),
-        bindingOp.byte_length(), bindingOp.dynamic_dims());
+        bindingOp.byte_length(), bindingOp.dynamic_dims(),
+        bindingOp.alignmentAttr());
     return success();
   }
 };
diff --git a/iree/compiler/Dialect/HAL/IR/HALOps.cpp b/iree/compiler/Dialect/HAL/IR/HALOps.cpp
index 09f0060..c87845a 100644
--- a/iree/compiler/Dialect/HAL/IR/HALOps.cpp
+++ b/iree/compiler/Dialect/HAL/IR/HALOps.cpp
@@ -16,7 +16,9 @@
 #include "mlir/IR/Attributes.h"
 #include "mlir/IR/Builders.h"
 #include "mlir/IR/BuiltinTypes.h"
+#include "mlir/IR/Matchers.h"
 #include "mlir/IR/OpImplementation.h"
+#include "mlir/IR/PatternMatch.h"
 #include "mlir/IR/SymbolTable.h"
 #include "mlir/IR/TypeUtilities.h"
 
@@ -1124,6 +1126,58 @@
                                          builder);
 }
 
+// TODO(benvanik): share with align op folder and analysis.
+// May need an interface for querying the alignment from ops that can carry it.
+
+// Tries to find the alignment of the given |value| based on either the IR
+// structure or annotations.
+static llvm::Optional<APInt> lookupValueOrAlignment(Value value) {
+  APInt constantValue;
+  if (matchPattern(value, m_ConstantInt(&constantValue))) {
+    // Value is constant and we can just treat that as if it were an alignment.
+    return constantValue;
+  }
+
+  auto op = value.getDefiningOp();
+  if (auto loadOp = dyn_cast_or_null<IREE::HAL::InterfaceLoadConstantOp>(op)) {
+    // Push constants have an optional value alignment.
+    auto alignment = loadOp.alignment();
+    if (alignment.hasValue()) return alignment;
+  }
+
+  // TODO(benvanik): more searching.
+  return llvm::None;
+}
+
+llvm::Align InterfaceBindingSubspanOp::calculateAlignment() {
+  // If we can't calculate an alignment we fall back to the natural alignment of
+  // the element type (for example, a memref<?xi32> is known to be at least
+  // 4-byte aligned).
+  llvm::Align naturalAlignment(1);
+  auto resultType = getType();
+  if (auto shapedType = resultType.dyn_cast<ShapedType>()) {
+    naturalAlignment = llvm::Align(
+        IREE::Util::getRoundedElementByteWidth(shapedType.getElementType()));
+  }
+
+  // If the binding has no assigned alignment we fall back to natural alignment.
+  auto bindingAlignmentInt = alignment();
+  if (!bindingAlignmentInt) return naturalAlignment;
+  auto bindingAlignment =
+      llvm::Align(bindingAlignmentInt.getValue().getZExtValue());
+
+  // Try to get the alignment of the byte offset. If it's a constant then we can
+  // find a common alignment between it and the base and otherwise we need to
+  // try to infer the alignment from the IR - otherwise we fall back.
+  auto offsetOrAlignment = lookupValueOrAlignment(byte_offset());
+  if (!offsetOrAlignment.hasValue()) return naturalAlignment;
+
+  // Compute the common alignment between that of the binding base and that of
+  // the byte offset.
+  return llvm::commonAlignment(bindingAlignment,
+                               offsetOrAlignment->getZExtValue());
+}
+
 //===----------------------------------------------------------------------===//
 // hal.interface.workgroup.*
 //===----------------------------------------------------------------------===//
diff --git a/iree/compiler/Dialect/HAL/IR/HALOps.h b/iree/compiler/Dialect/HAL/IR/HALOps.h
index 8ec603a..c16c8a7 100644
--- a/iree/compiler/Dialect/HAL/IR/HALOps.h
+++ b/iree/compiler/Dialect/HAL/IR/HALOps.h
@@ -15,6 +15,7 @@
 #include "iree/compiler/Dialect/Shape/IR/ShapeTypes.h"
 #include "iree/compiler/Dialect/Util/IR/UtilOps.h"
 #include "iree/compiler/Dialect/Util/IR/UtilTraits.h"
+#include "llvm/Support/Alignment.h"
 #include "mlir/IR/Attributes.h"
 #include "mlir/IR/BuiltinOps.h"
 #include "mlir/IR/BuiltinTypes.h"
diff --git a/iree/compiler/Dialect/HAL/IR/HALOps.td b/iree/compiler/Dialect/HAL/IR/HALOps.td
index 4ce7380..95fd802 100644
--- a/iree/compiler/Dialect/HAL/IR/HALOps.td
+++ b/iree/compiler/Dialect/HAL/IR/HALOps.td
@@ -2436,10 +2436,23 @@
     Loads a scalar constant value from an executable IO push constant block.
     The value will be loaded from the given constant offset and will be
     bitcast (possibly with truncation or zero-extension) to the result type.
+
+    An optional alignment indicates the byte alignment of potential values for
+    the constant when it could be determined from analysis. If omitted the value
+    may be anything and its interpretation is up to the usage. This is intended
+    to provide pointer alignment-like semantics to constants that are used to
+    index into binding resources.
+
+    An optional set of values indicates all possible values that can be passed
+    to the constant from all dispatch sites in the program. If omitted the value
+    may be from an unanalyzable source (outside of the program, indirect, etc)
+    and must be assumed to have any value.
   }];
 
   let arguments = (ins
-    HAL_HostSizeAttr:$offset
+    HAL_HostSizeAttr:$offset,
+    OptionalAttr<IndexAttr>:$alignment,
+    OptionalAttr<ArrayAttr>:$values
   );
   let results = (outs
     HAL_PrimitiveType:$result
@@ -2457,14 +2470,21 @@
   ]> {
   let summary = [{returns an alias to a subspan of interface binding data}];
   let description = [{
-    // TODO(benvanik): add description
+    Returns a subspan of an interface binding storage buffer in a generic type.
+    The exact shape, type, and alignment of the returned type are defined by
+    the result type (tensor, memref, etc).
+
+    An optional alignment indicates the byte alignment of the base binding
+    resource. Note that the byte offset is added to the base and the alignment
+    will be the minimum of the two.
   }];
 
   let arguments = (ins
     SymbolRefAttr:$binding,
     HAL_DeviceSize:$byte_offset,
     Optional<HAL_DeviceSize>:$byte_length,
-    HAL_ShapeDynamicDims:$dynamic_dims
+    HAL_ShapeDynamicDims:$dynamic_dims,
+    OptionalAttr<IndexAttr>:$alignment
   );
   let results = (outs
     Res<AnyType, "", [MemAlloc]>:$result
@@ -2484,6 +2504,12 @@
     /// Returns the hal.interface.binding op associated with this op.
     /// Returns null op if not found.
     IREE::HAL::InterfaceBindingOp queryBindingOp();
+
+    // Attempts to calculate an alignment of the final subspan offset in the
+    // parent storage buffer. This is a combination of both the binding
+    // alignment and the byte offset and may be as small as the natural
+    // alignment of the element type being accessed.
+    llvm::Align calculateAlignment();
   }];
 }
 
diff --git a/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces.cpp b/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces.cpp
index 7283b88..5351f66 100644
--- a/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces.cpp
+++ b/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces.cpp
@@ -579,7 +579,7 @@
       case RegionOperand::Type::PUSH_CONSTANT: {
         auto loadOp = entryBuilder.create<IREE::HAL::InterfaceLoadConstantOp>(
             clonedFuncOp.getLoc(), blockArg.getType(),
-            APInt(64, value.pushConstantOrdinal));
+            APInt(64, value.pushConstantOrdinal), nullptr, nullptr);
         blockArg.replaceAllUsesWith(loadOp);
         break;
       }
@@ -624,7 +624,8 @@
         auto subspanOp =
             entryBuilder.create<IREE::HAL::InterfaceBindingSubspanOp>(
                 clonedFuncOp.getLoc(), blockArgType, bindingSymRefAttr,
-                /*byte_offset=*/offset, /*byte_length=*/Value{}, dynamicDims);
+                /*byte_offset=*/offset, /*byte_length=*/Value{}, dynamicDims,
+                /*alignment=*/nullptr);
         blockArg.replaceAllUsesWith(subspanOp);
 
         break;
diff --git a/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces2.cpp b/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces2.cpp
index 95c7855..6189aae 100644
--- a/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces2.cpp
+++ b/iree/compiler/Dialect/HAL/Transforms/MaterializeInterfaces2.cpp
@@ -163,15 +163,20 @@
 }
 
 // Converts the usage of the given primitive |arg| to interface methods.
-static void convertOperandUsage(BlockArgument arg, unsigned pushConstantIdx,
-                                OpBuilder &builder) {
+static void convertOperandUsage(mlir::FuncOp sourceFuncOp, BlockArgument arg,
+                                unsigned pushConstantIdx, OpBuilder &builder) {
+  auto alignmentAttr = sourceFuncOp.getArgAttrOfType<IntegerAttr>(
+      arg.getArgNumber(), "stream.alignment");
+  auto valuesAttr = sourceFuncOp.getArgAttrOfType<ArrayAttr>(arg.getArgNumber(),
+                                                             "stream.values");
   auto loadOp = builder.create<IREE::HAL::InterfaceLoadConstantOp>(
-      arg.getLoc(), arg.getType(), APInt(64, pushConstantIdx));
+      arg.getLoc(), arg.getType(), builder.getIndexAttr(pushConstantIdx),
+      alignmentAttr, valuesAttr);
   arg.replaceAllUsesWith(loadOp);
 }
 
 // Converts the usage of the given !stream.binding |arg| to interface methods.
-static void convertBindingUsage(BlockArgument arg,
+static void convertBindingUsage(mlir::FuncOp sourceFuncOp, BlockArgument arg,
                                 IREE::HAL::InterfaceOp interfaceOp,
                                 IREE::HAL::InterfaceBindingOp bindingOp) {
   if (arg.use_empty()) return;  // no-op
@@ -179,11 +184,14 @@
     auto oldOp = dyn_cast<IREE::Stream::BindingSubspanOp>(use.getOwner());
     assert(oldOp && "bindings are only usable by stream.binding.subspan");
     OpBuilder builder(oldOp);
+    auto alignmentAttr = sourceFuncOp.getArgAttrOfType<IntegerAttr>(
+        arg.getArgNumber(), "stream.alignment");
     auto newOp = builder.create<IREE::HAL::InterfaceBindingSubspanOp>(
         oldOp.getLoc(), oldOp.getType(),
         SymbolRefAttr::get(interfaceOp.sym_nameAttr(),
                            {SymbolRefAttr::get(bindingOp)}),
-        oldOp.byte_offset(), /*byte_length=*/Value{}, oldOp.dynamic_dims());
+        oldOp.byte_offset(), /*byte_length=*/Value{}, oldOp.dynamic_dims(),
+        alignmentAttr);
     oldOp.replaceAllUsesWith(newOp.result());
     oldOp.erase();
   }
@@ -210,13 +218,13 @@
   for (auto arg : entryBlock->getArguments()) {
     if (!arg.getType().isa<IREE::Stream::BindingType>()) {
       // TODO(benvanik): symbolic push constant indices.
-      convertOperandUsage(arg, operandIdx++, entryBuilder);
+      convertOperandUsage(sourceFuncOp, arg, operandIdx++, entryBuilder);
     }
   }
   unsigned bindingIdx = 0;
   for (auto arg : entryBlock->getArguments()) {
     if (arg.getType().isa<IREE::Stream::BindingType>()) {
-      convertBindingUsage(arg, interface.op,
+      convertBindingUsage(sourceFuncOp, arg, interface.op,
                           interface.resourceBindings[bindingIdx++]);
     }
   }
diff --git a/iree/compiler/Dialect/Stream/IR/StreamOps.cpp b/iree/compiler/Dialect/Stream/IR/StreamOps.cpp
index 8193e83..df5cf54 100644
--- a/iree/compiler/Dialect/Stream/IR/StreamOps.cpp
+++ b/iree/compiler/Dialect/Stream/IR/StreamOps.cpp
@@ -1553,6 +1553,23 @@
   return map;
 }
 
+// static
+SmallVector<unsigned> CmdDispatchOp::makeResourceToArgMap(mlir::FuncOp funcOp) {
+  unsigned operandCount = llvm::count_if(
+      funcOp.getArgumentTypes(),
+      [](Type type) { return type.isa<IREE::Stream::BindingType>(); });
+  SmallVector<unsigned> map(operandCount);
+  unsigned operandIdx = 0;
+  for (auto it : llvm::enumerate(funcOp.getArgumentTypes())) {
+    unsigned argIdx = it.index();
+    auto argType = it.value();
+    if (argType.isa<IREE::Stream::BindingType>()) {
+      map[operandIdx++] = argIdx;
+    }
+  }
+  return map;
+}
+
 //===----------------------------------------------------------------------===//
 // stream.cmd.execute
 //===----------------------------------------------------------------------===//
diff --git a/iree/compiler/Dialect/Stream/IR/StreamOps.td b/iree/compiler/Dialect/Stream/IR/StreamOps.td
index 07b550a..2cdb2fe 100644
--- a/iree/compiler/Dialect/Stream/IR/StreamOps.td
+++ b/iree/compiler/Dialect/Stream/IR/StreamOps.td
@@ -2188,6 +2188,8 @@
 
     // Builds a map of operand index to argument index.
     static SmallVector<unsigned> makeOperandToArgMap(mlir::FuncOp funcOp);
+    // Builds a map of resource to argument index of the corresponding binding.
+    static SmallVector<unsigned> makeResourceToArgMap(mlir::FuncOp funcOp);
   }];
 
   let verifier = [{ return verifyOp(*this); }];
diff --git a/iree/compiler/Dialect/Stream/Transforms/AnnotateDispatchArguments.cpp b/iree/compiler/Dialect/Stream/Transforms/AnnotateDispatchArguments.cpp
new file mode 100644
index 0000000..16aff8f
--- /dev/null
+++ b/iree/compiler/Dialect/Stream/Transforms/AnnotateDispatchArguments.cpp
@@ -0,0 +1,468 @@
+// Copyright 2021 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 <memory>
+#include <utility>
+
+#include "iree/compiler/Dialect/Stream/IR/StreamOps.h"
+#include "iree/compiler/Dialect/Stream/Transforms/PassDetail.h"
+#include "iree/compiler/Dialect/Stream/Transforms/Passes.h"
+#include "iree/compiler/Dialect/Util/Analysis/DFX/Solver.h"
+#include "iree/compiler/Dialect/Util/Analysis/DFX/State.h"
+#include "iree/compiler/Dialect/Util/Analysis/Explorer.h"
+#include "llvm/ADT/APInt.h"
+#include "llvm/Support/Alignment.h"
+#include "llvm/Support/Debug.h"
+#include "mlir/Dialect/StandardOps/IR/Ops.h"
+#include "mlir/IR/AsmState.h"
+#include "mlir/IR/Attributes.h"
+#include "mlir/IR/Builders.h"
+#include "mlir/IR/BuiltinTypes.h"
+#include "mlir/IR/Diagnostics.h"
+#include "mlir/IR/Matchers.h"
+#include "mlir/Pass/Pass.h"
+
+#define DEBUG_TYPE "iree-stream-annotate-dispatch-arguments"
+
+namespace mlir {
+namespace iree_compiler {
+namespace IREE {
+namespace Stream {
+namespace {
+
+//===----------------------------------------------------------------------===//
+// Analysis state
+//===----------------------------------------------------------------------===//
+
+// TODO(benvanik): move to Util/Analysis/ as this would be useful in other
+// passes as well and only depends on util.align and upstream ops.
+
+class ValuePVS : public DFX::StateWrapper<DFX::PotentialConstantIntValuesState,
+                                          DFX::ValueElement> {
+ public:
+  using BaseType = DFX::StateWrapper<DFX::PotentialConstantIntValuesState,
+                                     DFX::ValueElement>;
+
+  static ValuePVS &createForPosition(const Position &pos, DFX::Solver &solver) {
+    return *(new (solver.getAllocator()) ValuePVS(pos));
+  }
+
+  const std::string getName() const override { return "ValuePVS"; }
+  const void *getID() const override { return &ID; }
+  static bool classof(const DFX::AbstractElement *element) {
+    return (element->getID() == &ID);
+  }
+  static const char ID;
+
+  const std::string getAsStr() const override {
+    std::string str;
+    llvm::raw_string_ostream sstream(str);
+    sstream << "pvs: ";
+    if (isValidState()) {
+      sstream << "[";
+      if (isUndefContained()) {
+        sstream << "undef, ";
+      }
+      llvm::interleaveComma(getAssumedSet(), sstream,
+                            [&](APInt value) { value.print(sstream, false); });
+      sstream << "]";
+    } else {
+      sstream << "(invalid)";
+    }
+    sstream.flush();
+    return str;
+  }
+
+ private:
+  explicit ValuePVS(const Position &pos) : BaseType(pos) {}
+
+  void initializeValue(Value value, DFX::Solver &solver) override {
+    APInt staticValue;
+    if (matchPattern(value, m_ConstantInt(&staticValue))) {
+      unionAssumed(staticValue);
+      indicateOptimisticFixpoint();
+    }
+  }
+
+  ChangeStatus updateValue(Value value, DFX::Solver &solver) override {
+    StateType newState;
+    if (solver.getExplorer().walkDefiningOps(value, [&](OpResult result) {
+          APInt staticValue;
+          if (matchPattern(result, m_ConstantInt(&staticValue))) {
+            newState.unionAssumed(staticValue);
+            return WalkResult::advance();
+          }
+
+          // TODO(benvanik): more ops supported for joining. We could for
+          // example walk the lhs/rhs of elementwise ops and perform the set
+          // operations (so addi %lhs, %rhs could produce a PVS of all of %lhs
+          // summed to all of %rhs). May not be worth it, though.
+          // TODO(benvanik): move select op walking to the explorer.
+          if (auto selectOp =
+                  dyn_cast<mlir::SelectOp>(result.getDefiningOp())) {
+            auto lhs = solver.getElementFor<ValuePVS>(
+                *this, Position::forValue(selectOp.true_value()),
+                DFX::Resolution::REQUIRED);
+            auto rhs = solver.getElementFor<ValuePVS>(
+                *this, Position::forValue(selectOp.false_value()),
+                DFX::Resolution::REQUIRED);
+            if (!lhs.isValidState() || !rhs.isValidState()) {
+              newState.unionAssumedWithUndef();
+              newState.indicatePessimisticFixpoint();
+            } else {
+              newState.unionAssumed(lhs);
+              newState.unionAssumed(rhs);
+            }
+            return WalkResult::advance();
+          }
+
+          // Some other dynamic value we can't analyze (yet).
+          newState.unionAssumedWithUndef();
+          newState.indicatePessimisticFixpoint();
+          return WalkResult::advance();
+        }) == TraversalResult::INCOMPLETE) {
+      newState.unionAssumedWithUndef();
+      newState.indicatePessimisticFixpoint();
+    }
+    return DFX::clampStateAndIndicateChange(getState(), newState);
+  }
+
+  friend class DFX::Solver;
+};
+const char ValuePVS::ID = 0;
+
+static constexpr uint64_t kMaximumAlignment = 1ull << 32;
+
+using AlignmentStateType = DFX::IncIntegerState<uint64_t, kMaximumAlignment, 1>;
+class ValueAlignment
+    : public DFX::StateWrapper<AlignmentStateType, DFX::ValueElement> {
+ public:
+  using BaseType = DFX::StateWrapper<AlignmentStateType, DFX::ValueElement>;
+
+  static ValueAlignment &createForPosition(const Position &pos,
+                                           DFX::Solver &solver) {
+    return *(new (solver.getAllocator()) ValueAlignment(pos));
+  }
+
+  llvm::MaybeAlign getAssumedAlignment() const {
+    return llvm::MaybeAlign(getAssumed());
+  }
+
+  llvm::MaybeAlign getKnownAlignment() const {
+    return llvm::MaybeAlign(getKnown());
+  }
+
+  const std::string getName() const override { return "ValueAlignment"; }
+  const void *getID() const override { return &ID; }
+  static bool classof(const DFX::AbstractElement *element) {
+    return (element->getID() == &ID);
+  }
+  static const char ID;
+
+  const std::string getAsStr() const override {
+    return std::string("alignment: ") +
+           std::to_string(getAssumedAlignment().valueOrOne().value());
+  }
+
+ private:
+  explicit ValueAlignment(const Position &pos) : BaseType(pos) {}
+
+  void initializeValue(Value value, DFX::Solver &solver) override {
+    if (!value.getType().isIndex()) {
+      indicatePessimisticFixpoint();
+      return;
+    }
+  }
+
+  static llvm::MaybeAlign computeAlignment(const ValuePVS::SetTy &set) {
+    if (set.empty()) return llvm::MaybeAlign();
+    llvm::MaybeAlign alignment;
+    for (auto value : set) {
+      APInt valueDivisor = (value & (~(value - 1)));
+      alignment = llvm::commonAlignment(
+          alignment, llvm::MaybeAlign(valueDivisor.getZExtValue()));
+    }
+    return alignment;
+  }
+
+  ChangeStatus updateValue(Value value, DFX::Solver &solver) override {
+    StateType newState = getState();
+
+    // If we can get a full potential value set then we can derive an alignment
+    // from that.
+    auto pvs = solver.getElementFor<ValuePVS>(*this, Position::forValue(value),
+                                              DFX::Resolution::OPTIONAL);
+    if (pvs.isValidState() && !pvs.isUndefContained()) {
+      auto alignment = computeAlignment(pvs.getAssumedSet());
+      if (alignment.hasValue()) {
+        newState.takeAssumedMinimum(alignment.valueOrOne().value());
+        newState.indicateOptimisticFixpoint();
+      }
+    }
+
+    if (!newState.isAtFixpoint()) {
+      // Scan IR to see if we can infer the alignment.
+      // TODO(benvanik): walk math ops (like muli) to peek through to alignments
+      // of inputs. For now we just look for util.align only. We should also
+      // be able to look through casts/exts/etc and affine.apply.
+      if (solver.getExplorer().walkDefiningOps(value, [&](OpResult result) {
+            if (auto alignOp =
+                    dyn_cast<IREE::Util::AlignOp>(result.getDefiningOp())) {
+              auto alignment = solver.getElementFor<ValueAlignment>(
+                  *this, Position::forValue(alignOp.alignment()),
+                  DFX::Resolution::REQUIRED);
+              newState ^= alignment;
+            }
+            return WalkResult::advance();
+          }) == TraversalResult::INCOMPLETE) {
+        newState.indicatePessimisticFixpoint();
+      }
+    }
+
+    return DFX::clampStateAndIndicateChange(getState(), newState);
+  }
+
+  friend class DFX::Solver;
+};
+const char ValueAlignment::ID = 0;
+
+class ArgumentAnalysis {
+ public:
+  explicit ArgumentAnalysis(Operation *rootOp)
+      : explorer(rootOp, TraversalAction::SHALLOW),
+        solver(explorer, allocator) {
+    explorer.setOpAction<IREE::Util::InitializerOp>(TraversalAction::RECURSE);
+    explorer.setOpAction<mlir::FuncOp>(TraversalAction::RECURSE);
+    explorer.setDialectAction<IREE::Stream::StreamDialect>(
+        TraversalAction::RECURSE);
+    // Ignore the contents of executables (linalg goo, etc).
+    explorer.setOpAction<IREE::Stream::ExecutableOp>(TraversalAction::IGNORE);
+    explorer.initialize();
+
+    // Find all dispatches and bucket by their target entry point.
+    rootOp->walk([&](IREE::Stream::CmdDispatchOp dispatchOp) {
+      auto exportOp = explorer.getSymbolTables().lookupNearestSymbolFrom(
+          dispatchOp, dispatchOp.entry_point());
+      entryDispatchMap[exportOp].push_back(dispatchOp);
+    });
+  }
+
+  // Runs analysis and populates the state cache.
+  // May fail if analysis cannot be completed due to unsupported or unknown IR.
+  LogicalResult run() {
+    // Seed all dispatch arguments we want to analyze.
+    for (auto it : entryDispatchMap) {
+      for (auto dispatchOp : it.second) {
+        for (auto operand : dispatchOp.operands()) {
+          solver.getOrCreateElementFor<ValuePVS>(Position::forValue(operand));
+          solver.getOrCreateElementFor<ValueAlignment>(
+              Position::forValue(operand));
+        }
+        for (auto resourceOffset : dispatchOp.resource_offsets()) {
+          solver.getOrCreateElementFor<ValueAlignment>(
+              Position::forValue(resourceOffset));
+        }
+      }
+    }
+
+    // Run solver to completion.
+    return solver.run();
+  }
+
+  // Returns a list of dispatch sites in arbitrary order to the given
+  // |exportOp|.
+  ArrayRef<IREE::Stream::CmdDispatchOp> getDispatchSites(
+      IREE::Stream::ExecutableExportOp exportOp) {
+    auto it = entryDispatchMap.find(exportOp);
+    if (it == entryDispatchMap.end()) return {};
+    return it->second;
+  }
+
+  // Returns the minimum alignment for the given |value| or None if it could not
+  // be analyzed and natural alignment should be assumed.
+  llvm::MaybeAlign getAlignmentFor(Value value) {
+    auto element =
+        solver.lookupElementFor<ValueAlignment>(Position::forValue(value));
+    if (!element) return llvm::MaybeAlign();
+    return element->getAssumedAlignment();
+  }
+
+  // TODO(benvanik): replace these with dedicated
+  // ArgumentAlignment/ResourceOffsetAlignment state that does this unioning as
+  // part of the solver. It's not strictly required as this is unidirectional
+  // (the alignment of the export arguments is dictated by the dispatch sites
+  // and not the other way around) but would be cleaner.
+
+  // Returns the potential constant values across all dispatch sites to
+  // |exportOp| for the operand at |operandIdx|.
+  DFX::PotentialConstantIntValuesState getOperandPVS(
+      IREE::Stream::ExecutableExportOp exportOp, unsigned operandIdx) {
+    DFX::PotentialConstantIntValuesState state;
+    for (auto dispatchOp : getDispatchSites(exportOp)) {
+      auto element = solver.lookupElementFor<ValuePVS>(
+          Position::forValue(dispatchOp.operands()[operandIdx]));
+      if (!element) {
+        state.unionAssumedWithUndef();
+        state.indicatePessimisticFixpoint();
+        break;
+      }
+      state ^= element->getState();
+    }
+    return state;
+  }
+
+  // Returns the minimum alignment across all dispatch sites to |exportOp| for
+  // the operand at |operandIdx|.
+  llvm::MaybeAlign getOperandAlignment(
+      IREE::Stream::ExecutableExportOp exportOp, unsigned operandIdx) {
+    llvm::MaybeAlign alignment;
+    for (auto dispatchOp : getDispatchSites(exportOp)) {
+      auto element = solver.lookupElementFor<ValueAlignment>(
+          Position::forValue(dispatchOp.operands()[operandIdx]));
+      if (!element || !element->isValidState()) return llvm::MaybeAlign();
+      alignment =
+          llvm::commonAlignment(alignment, element->getAssumedAlignment());
+    }
+    if (alignment.valueOrOne().value() == kMaximumAlignment) {
+      return llvm::MaybeAlign();
+    }
+    return alignment;
+  }
+
+  // Returns the minimum alignment across all dispatch sites to |exportOp| for
+  // the resource offset at |resourceIdx|.
+  llvm::MaybeAlign getResourceOffsetAlignment(
+      IREE::Stream::ExecutableExportOp exportOp, unsigned resourceIdx) {
+    llvm::MaybeAlign alignment;
+    for (auto dispatchOp : getDispatchSites(exportOp)) {
+      auto element = solver.lookupElementFor<ValueAlignment>(
+          Position::forValue(dispatchOp.resource_offsets()[resourceIdx]));
+      if (!element || !element->isValidState()) return llvm::MaybeAlign();
+      alignment =
+          llvm::commonAlignment(alignment, element->getAssumedAlignment());
+    }
+    if (alignment.valueOrOne().value() == kMaximumAlignment) {
+      // Alignment is natural, which for resources means the base resource
+      // alignment.
+      auto configAttr = IREE::Stream::ResourceConfigAttr::lookup(exportOp);
+      return llvm::MaybeAlign(configAttr.getMinBufferOffsetAlignment());
+    }
+    return alignment;
+  }
+
+ private:
+  Explorer explorer;
+  llvm::BumpPtrAllocator allocator;
+  DFX::Solver solver;
+
+  DenseMap<Operation *, SmallVector<IREE::Stream::CmdDispatchOp>>
+      entryDispatchMap;
+};
+
+//===----------------------------------------------------------------------===//
+// Per-dispatchable export argument annotation
+//===----------------------------------------------------------------------===//
+
+// Annotates |exportOp| (and its target function) with information derived from
+// all dispatch sites of that export.
+static void annotateExport(IREE::Stream::ExecutableOp executableOp,
+                           IREE::Stream::ExecutableExportOp exportOp,
+                           ArgumentAnalysis &analysis) {
+  auto *context = executableOp.getContext();
+
+  // Operands/resources on the func are in an arbitrary order; get maps that
+  // lets us go from dispatch site operand/resource to function argument.
+  auto funcOp = executableOp.getInnerModule().lookupSymbol<mlir::FuncOp>(
+      exportOp.function_refAttr());
+  auto operandToArgMap =
+      IREE::Stream::CmdDispatchOp::makeOperandToArgMap(funcOp);
+  auto resourceToArgMap =
+      IREE::Stream::CmdDispatchOp::makeResourceToArgMap(funcOp);
+
+  auto indexType = IndexType::get(context);
+
+  // Annotate operand arguments with their potential values and alignment.
+  for (unsigned operandIdx = 0; operandIdx < operandToArgMap.size();
+       ++operandIdx) {
+    unsigned argIdx = operandToArgMap[operandIdx];
+    auto argType = funcOp.getArgument(argIdx).getType();
+
+    auto pvs = analysis.getOperandPVS(exportOp, operandIdx);
+    if (pvs.isValidState() && !pvs.isUndefContained()) {
+      SmallVector<Attribute> potentialValues;
+      potentialValues.reserve(pvs.getAssumedSet().size());
+      for (auto value : pvs.getAssumedSet()) {
+        potentialValues.push_back(IntegerAttr::get(argType, value));
+      }
+      auto potentialValuesAttr = ArrayAttr::get(context, potentialValues);
+      funcOp.setArgAttr(argIdx, "stream.values", potentialValuesAttr);
+    }
+
+    if (argType.isIndex()) {
+      auto alignment = analysis.getOperandAlignment(exportOp, operandIdx);
+      if (alignment.hasValue()) {
+        uint64_t alignmentOrOne = alignment.valueOrOne().value();
+        funcOp.setArgAttr(argIdx, "stream.alignment",
+                          IntegerAttr::get(indexType, alignmentOrOne));
+      }
+    }
+  }
+
+  // Annotate binding arguments with their base alignment.
+  for (unsigned resourceIdx = 0; resourceIdx < resourceToArgMap.size();
+       ++resourceIdx) {
+    unsigned argIdx = resourceToArgMap[resourceIdx];
+    auto alignment = analysis.getResourceOffsetAlignment(exportOp, resourceIdx);
+    if (alignment.hasValue()) {
+      uint64_t alignmentOrOne = alignment.valueOrOne().value();
+      funcOp.setArgAttr(argIdx, "stream.alignment",
+                        IntegerAttr::get(indexType, alignmentOrOne));
+    }
+  }
+}
+
+//===----------------------------------------------------------------------===//
+// -iree-stream-specialize-dispatches
+//===----------------------------------------------------------------------===//
+
+class AnnotateDispatchArgumentsPass
+    : public AnnotateDispatchArgumentsBase<AnnotateDispatchArgumentsPass> {
+ public:
+  AnnotateDispatchArgumentsPass() = default;
+
+  void getDependentDialects(DialectRegistry &registry) const override {
+    registry.insert<IREE::Stream::StreamDialect>();
+  }
+
+  void runOnOperation() override {
+    // Perform argument value analysis.
+    ArgumentAnalysis analysis(getOperation());
+    if (failed(analysis.run())) {
+      return signalPassFailure();
+    }
+
+    // Annotate the exported dispatch functions.
+    for (auto executableOp :
+         getOperation().body().getOps<IREE::Stream::ExecutableOp>()) {
+      for (auto exportOp :
+           executableOp.getOps<IREE::Stream::ExecutableExportOp>()) {
+        annotateExport(executableOp, exportOp, analysis);
+      }
+    }
+  }
+};
+
+}  // namespace
+
+std::unique_ptr<OperationPass<mlir::ModuleOp>>
+createAnnotateDispatchArgumentsPass() {
+  return std::make_unique<AnnotateDispatchArgumentsPass>();
+}
+
+}  // namespace Stream
+}  // namespace IREE
+}  // namespace iree_compiler
+}  // namespace mlir
diff --git a/iree/compiler/Dialect/Stream/Transforms/BUILD b/iree/compiler/Dialect/Stream/Transforms/BUILD
index 1f9c461..7939480 100644
--- a/iree/compiler/Dialect/Stream/Transforms/BUILD
+++ b/iree/compiler/Dialect/Stream/Transforms/BUILD
@@ -15,6 +15,7 @@
 cc_library(
     name = "Transforms",
     srcs = [
+        "AnnotateDispatchArguments.cpp",
         "ConvertToStream.cpp",
         "ElideAsyncCopies.cpp",
         "EncodeTensors.cpp",
diff --git a/iree/compiler/Dialect/Stream/Transforms/CMakeLists.txt b/iree/compiler/Dialect/Stream/Transforms/CMakeLists.txt
index 4b6f32a..98c9a47 100644
--- a/iree/compiler/Dialect/Stream/Transforms/CMakeLists.txt
+++ b/iree/compiler/Dialect/Stream/Transforms/CMakeLists.txt
@@ -17,6 +17,7 @@
     "Passes.h"
     "Passes.h.inc"
   SRCS
+    "AnnotateDispatchArguments.cpp"
     "ConvertToStream.cpp"
     "ElideAsyncCopies.cpp"
     "EncodeTensors.cpp"
diff --git a/iree/compiler/Dialect/Stream/Transforms/Passes.cpp b/iree/compiler/Dialect/Stream/Transforms/Passes.cpp
index 8fb5f0f..8425eef 100644
--- a/iree/compiler/Dialect/Stream/Transforms/Passes.cpp
+++ b/iree/compiler/Dialect/Stream/Transforms/Passes.cpp
@@ -243,6 +243,15 @@
     // make partitioning the bindings easier. Note we need to update both the
     // dispatches and the dispatch function argument order.
   }
+
+  //----------------------------------------------------------------------------
+  // Annotations to aid future lowering pipelines
+  //----------------------------------------------------------------------------
+
+  // Annotate dispatch region arguments based on the operands passed at dispatch
+  // sites. This allows codegen to see the potential values for the operands
+  // when operating locally on executables.
+  passManager.addPass(IREE::Stream::createAnnotateDispatchArgumentsPass());
 }
 
 //===----------------------------------------------------------------------===//
diff --git a/iree/compiler/Dialect/Stream/Transforms/Passes.h b/iree/compiler/Dialect/Stream/Transforms/Passes.h
index 6ee8021..4af9377 100644
--- a/iree/compiler/Dialect/Stream/Transforms/Passes.h
+++ b/iree/compiler/Dialect/Stream/Transforms/Passes.h
@@ -124,6 +124,9 @@
 std::unique_ptr<OperationPass<mlir::ModuleOp>> createFuseDispatchBindingsPass();
 std::unique_ptr<OperationPass<mlir::ModuleOp>> createSpecializeDispatchesPass();
 
+std::unique_ptr<OperationPass<mlir::ModuleOp>>
+createAnnotateDispatchArgumentsPass();
+
 //===----------------------------------------------------------------------===//
 // Diagnostics
 //===----------------------------------------------------------------------===//
diff --git a/iree/compiler/Dialect/Stream/Transforms/Passes.td b/iree/compiler/Dialect/Stream/Transforms/Passes.td
index 8686fae..dfc4c89 100644
--- a/iree/compiler/Dialect/Stream/Transforms/Passes.td
+++ b/iree/compiler/Dialect/Stream/Transforms/Passes.td
@@ -176,6 +176,14 @@
   }];
 }
 
+def AnnotateDispatchArguments :
+    Pass<"iree-stream-annotate-dispatch-arguments", "mlir::ModuleOp"> {
+  let summary = "Annotates dispatch arguments with potential values derived from dispatch sites.";
+  let constructor = [{
+    mlir::iree_compiler::IREE::Stream::createAnnotateDispatchArgumentsPass()
+  }];
+}
+
 //===----------------------------------------------------------------------===//
 // Stream memoization
 //===----------------------------------------------------------------------===//
diff --git a/iree/compiler/Dialect/Stream/Transforms/test/BUILD b/iree/compiler/Dialect/Stream/Transforms/test/BUILD
index 6d217da..787319c 100644
--- a/iree/compiler/Dialect/Stream/Transforms/test/BUILD
+++ b/iree/compiler/Dialect/Stream/Transforms/test/BUILD
@@ -17,6 +17,7 @@
     name = "lit",
     srcs = enforce_glob(
         [
+            "annotate_dispatch_arguments.mlir",
             "convert_to_stream.mlir",
             "elide_async_copies.mlir",
             "encode_tensors.mlir",
diff --git a/iree/compiler/Dialect/Stream/Transforms/test/CMakeLists.txt b/iree/compiler/Dialect/Stream/Transforms/test/CMakeLists.txt
index 40905cf..354d5e5 100644
--- a/iree/compiler/Dialect/Stream/Transforms/test/CMakeLists.txt
+++ b/iree/compiler/Dialect/Stream/Transforms/test/CMakeLists.txt
@@ -14,6 +14,7 @@
   NAME
     lit
   SRCS
+    "annotate_dispatch_arguments.mlir"
     "convert_to_stream.mlir"
     "elide_async_copies.mlir"
     "encode_tensors.mlir"
diff --git a/iree/compiler/Dialect/Stream/Transforms/test/annotate_dispatch_arguments.mlir b/iree/compiler/Dialect/Stream/Transforms/test/annotate_dispatch_arguments.mlir
new file mode 100644
index 0000000..becbb9b
--- /dev/null
+++ b/iree/compiler/Dialect/Stream/Transforms/test/annotate_dispatch_arguments.mlir
@@ -0,0 +1,132 @@
+// RUN: iree-opt -split-input-file -iree-stream-annotate-dispatch-arguments %s | IreeFileCheck %s
+
+// Tests that operands are annotated with their potential values.
+// %arg0: cannot be annotated because it comes from outside the program.
+// %arg1: all values known, gets alignment being an index.
+// %arg2: all values known, no alignment (doesn't make sense for i1).
+// %arg3: don't yet support floats.
+
+// CHECK-LABEL: @annotatePotentialValuesEx
+stream.executable private @annotatePotentialValuesEx {
+  stream.executable.export public @dispatch
+  builtin.module  {
+    // CHECK: func @dispatch(
+    // CHECK-SAME: %arg0: i32,
+    // CHECK-SAME: %arg1: index {stream.alignment = 4 : index, stream.values = [20 : index, 40 : index]},
+    // CHECK-SAME: %arg2: i1 {stream.values = [true, false]},
+    // CHECK-SAME: %arg3: f32)
+    func @dispatch(%arg0: i32, %arg1: index, %arg2: i1, %arg3: f32) {
+      return
+    }
+  }
+}
+func @annotatePotentialValues(%arg0: i32) {
+  %c0 = arith.constant 0 : index
+  %c1 = arith.constant 1 : index
+  %c0_i32 = arith.constant 0 : i32
+  %c20 = arith.constant 20 : index
+  %c40 = arith.constant 40 : index
+  %false = arith.constant 0 : i1
+  %true = arith.constant 1 : i1
+  %c500 = arith.constant 500.0 : f32
+  %c600 = arith.constant 600.0 : f32
+  %alloc = stream.resource.alloc uninitialized : !stream.resource<transient>{%c1}
+  %result_timepoint = stream.cmd.execute with(%alloc as %capture: !stream.resource<transient>{%c1}) {
+    stream.cmd.dispatch @annotatePotentialValuesEx::@dispatch[%c1, %c1, %c1](%c0_i32, %c20, %false, %c500 : i32, index, i1, f32) {
+      rw %capture[%c0 for %c1] : !stream.resource<transient>{%c1}
+    }
+    stream.cmd.dispatch @annotatePotentialValuesEx::@dispatch[%c1, %c1, %c1](%arg0, %c40, %true, %c600 : i32, index, i1, f32) {
+      rw %capture[%c0 for %c1] : !stream.resource<transient>{%c1}
+    }
+  } => !stream.timepoint
+  return
+}
+
+// -----
+
+// Tests that index operands are analyzed for alignment.
+// %arg0: not analyzable as %arg0 comes from outside the program.
+// %arg1: all values aren't known but the util.align gives us what we need.
+// %arg2: all values are known but unaligned.
+
+// CHECK-LABEL: @annotateOperandAlignmentEx
+stream.executable private @annotateOperandAlignmentEx {
+  stream.executable.export public @dispatch
+  builtin.module  {
+    // CHECK: func @dispatch(
+    // CHECK-SAME: %arg0: index,
+    // CHECK-SAME: %arg1: index {stream.alignment = 16 : index},
+    // CHECK-SAME: %arg2: index {stream.values = [4096 : index, 4097 : index]})
+    func @dispatch(%arg0: index, %arg1: index, %arg2: index) {
+      return
+    }
+  }
+}
+func @annotateOperandAlignment(%arg0: index, %arg1: index) {
+  %c0 = arith.constant 0 : index
+  %c1 = arith.constant 1 : index
+  %c16 = arith.constant 16 : index
+  %c32 = arith.constant 32 : index
+  %c4096 = arith.constant 4096 : index
+  %c4097 = arith.constant 4097 : index
+  %aligned1 = util.align %arg1, %c16 : index
+  %alloc = stream.resource.alloc uninitialized : !stream.resource<transient>{%c1}
+  %result_timepoint = stream.cmd.execute with(%alloc as %capture: !stream.resource<transient>{%c1}) {
+    stream.cmd.dispatch @annotateOperandAlignmentEx::@dispatch[%c1, %c1, %c1](%arg0, %c32, %c4097 : index, index, index) {
+      rw %capture[%c0 for %c1] : !stream.resource<transient>{%c1}
+    }
+    stream.cmd.dispatch @annotateOperandAlignmentEx::@dispatch[%c1, %c1, %c1](%c16, %aligned1, %c4096 : index, index, index) {
+      rw %capture[%c0 for %c1] : !stream.resource<transient>{%c1}
+    }
+  } => !stream.timepoint
+  return
+}
+
+// -----
+
+// Tests that resource offset alignment gets tagged on binding arguments.
+// %arg0: all values known (including 0), max alignment.
+// %arg1: value comes from outside the program so no alignment.
+// %arg2: all values known.
+// %arg3: util.align provides info required for external value.
+
+// CHECK-LABEL: @annotateBindingAlignmentEx
+stream.executable private @annotateBindingAlignmentEx {
+  stream.executable.export public @dispatch
+  builtin.module  {
+    // CHECK: func @dispatch(
+    // CHECK-SAME: %arg0: !stream.binding {stream.alignment = 64 : index},
+    // CHECK-SAME: %arg1: !stream.binding,
+    // CHECK-SAME: %arg2: !stream.binding {stream.alignment = 8 : index},
+    // CHECK-SAME: %arg3: !stream.binding {stream.alignment = 16 : index})
+    func @dispatch(%arg0: !stream.binding, %arg1: !stream.binding, %arg2: !stream.binding, %arg3: !stream.binding) {
+      return
+    }
+  }
+}
+func @annotateBindingAlignment(%arg0: index, %arg1: index) {
+  %c0 = arith.constant 0 : index
+  %c1 = arith.constant 1 : index
+  %c8 = arith.constant 8 : index
+  %c16 = arith.constant 16 : index
+  %c32 = arith.constant 32 : index
+  %c48 = arith.constant 48 : index
+  %c64 = arith.constant 64 : index
+  %aligned1 = util.align %arg1, %c16 : index
+  %alloc = stream.resource.alloc uninitialized : !stream.resource<transient>{%c64}
+  %result_timepoint = stream.cmd.execute with(%alloc as %capture: !stream.resource<transient>{%c64}) {
+    stream.cmd.dispatch @annotateBindingAlignmentEx::@dispatch[%c1, %c1, %c1] {
+      rw %capture[%c0 for %c8] : !stream.resource<transient>{%c64},
+      rw %capture[%arg0 for %c8] : !stream.resource<transient>{%c64},
+      rw %capture[%c8 for %c8] : !stream.resource<transient>{%c64},
+      rw %capture[%c32 for %c8] : !stream.resource<transient>{%c64}
+    }
+    stream.cmd.dispatch @annotateBindingAlignmentEx::@dispatch[%c1, %c1, %c1] {
+      rw %capture[%c64 for %c8] : !stream.resource<transient>{%c64},
+      rw %capture[%c1 for %c8] : !stream.resource<transient>{%c64},
+      rw %capture[%c16 for %c8] : !stream.resource<transient>{%c64},
+      rw %capture[%aligned1 for %c8] : !stream.resource<transient>{%c64}
+    }
+  } => !stream.timepoint
+  return
+}
diff --git a/iree/compiler/Dialect/Util/IR/UtilTypes.h b/iree/compiler/Dialect/Util/IR/UtilTypes.h
index d647eb3..219170d 100644
--- a/iree/compiler/Dialect/Util/IR/UtilTypes.h
+++ b/iree/compiler/Dialect/Util/IR/UtilTypes.h
@@ -187,6 +187,22 @@
   return align(value, alignment.getZExtValue());
 }
 
+// Returns the number of bytes an element of the given type occupies in memory.
+// This is in the default dense conversion to machine words where sizes must be
+// powers of two aligned to bytes.
+//
+// Example:
+//   getRoundedElementByteWidth(i1) = 1
+//   getRoundedElementByteWidth(i23) = 4
+//   getRoundedElementByteWidth(i32) = 4
+//   getRoundedElementByteWidth(bf16) = 2
+static inline int32_t getRoundedElementByteWidth(Type type) {
+  // Round up to 8-bit aligned bytes.
+  unsigned byteAligned = (type.getIntOrFloatBitWidth() + 8 - 1) / 8;
+  // Round up to the next power of two (unless already a power of two).
+  return llvm::PowerOf2Ceil(byteAligned);
+}
+
 }  // namespace Util
 }  // namespace IREE
 }  // namespace iree_compiler