[BACKEND] Make flash attention forward pass work (#928)
This also simplifies BroadcastOp codegen
This commit is contained in:
@@ -573,12 +573,11 @@ public:
|
||||
emitOffsetForMmaLayoutV2(const MmaEncodingAttr &mmaLayout,
|
||||
ArrayRef<int64_t> shape) const {
|
||||
SmallVector<SmallVector<unsigned>> ret;
|
||||
|
||||
for (unsigned i = 0; i < shape[0]; i += getShapePerCTA(mmaLayout)[0]) {
|
||||
for (unsigned j = 0; j < shape[1]; j += getShapePerCTA(mmaLayout)[1]) {
|
||||
ret.push_back({i, j});
|
||||
ret.push_back({i, j + 1});
|
||||
}
|
||||
for (unsigned j = 0; j < shape[1]; j += getShapePerCTA(mmaLayout)[1]) {
|
||||
ret.push_back({i + 8, j});
|
||||
ret.push_back({i + 8, j + 1});
|
||||
}
|
||||
@@ -645,6 +644,23 @@ public:
|
||||
return multiDimIdx;
|
||||
}
|
||||
|
||||
|
||||
struct SmallVectorKeyInfo {
|
||||
static unsigned getHashValue(const SmallVector<unsigned> &key) {
|
||||
return llvm::hash_combine_range(key.begin(), key.end());
|
||||
}
|
||||
static bool isEqual(const SmallVector<unsigned> &lhs,
|
||||
const SmallVector<unsigned> &rhs) {
|
||||
return lhs == rhs;
|
||||
}
|
||||
static SmallVector<unsigned> getEmptyKey() {
|
||||
return SmallVector<unsigned>();
|
||||
}
|
||||
static SmallVector<unsigned> getTombstoneKey() {
|
||||
return {std::numeric_limits<unsigned>::max()};
|
||||
}
|
||||
};
|
||||
|
||||
SmallVector<SmallVector<Value>>
|
||||
emitIndicesForSliceLayout(Location loc, ConversionPatternRewriter &rewriter,
|
||||
const SliceEncodingAttr &sliceLayout,
|
||||
@@ -652,15 +668,15 @@ public:
|
||||
auto parent = sliceLayout.getParent();
|
||||
unsigned dim = sliceLayout.getDim();
|
||||
size_t rank = shape.size();
|
||||
auto paddedIndices =
|
||||
auto parentIndices =
|
||||
emitIndices(loc, rewriter, parent, sliceLayout.paddedShape(shape));
|
||||
unsigned numIndices = paddedIndices.size();
|
||||
SmallVector<SmallVector<Value>> resultIndices(numIndices);
|
||||
for (unsigned i = 0; i < numIndices; ++i)
|
||||
for (unsigned d = 0; d < rank + 1; ++d)
|
||||
if (d != dim)
|
||||
resultIndices[i].push_back(paddedIndices[i][d]);
|
||||
|
||||
unsigned numIndices = parentIndices.size();
|
||||
SmallVector<SmallVector<Value>> resultIndices;
|
||||
for (unsigned i = 0; i < numIndices; ++i){
|
||||
SmallVector<Value> indices = parentIndices[i];
|
||||
indices.erase(indices.begin() + dim);
|
||||
resultIndices.push_back(indices);
|
||||
}
|
||||
return resultIndices;
|
||||
}
|
||||
|
||||
@@ -1219,92 +1235,24 @@ struct BroadcastOpConversion
|
||||
unsigned rank = srcTy.getRank();
|
||||
assert(rank == resultTy.getRank());
|
||||
auto order = triton::gpu::getOrder(srcLayout);
|
||||
|
||||
SmallVector<int64_t> srcLogicalShape(2 * rank);
|
||||
SmallVector<unsigned> srcLogicalOrder(2 * rank);
|
||||
SmallVector<int64_t> resultLogicalShape(2 * rank);
|
||||
SmallVector<unsigned> broadcastDims;
|
||||
for (unsigned d = 0; d < rank; ++d) {
|
||||
unsigned resultShapePerCTA =
|
||||
triton::gpu::getSizePerThread(resultLayout)[d] *
|
||||
triton::gpu::getThreadsPerWarp(resultLayout)[d] *
|
||||
triton::gpu::getWarpsPerCTA(resultLayout)[d];
|
||||
int64_t numCtas = ceil<unsigned>(resultShape[d], resultShapePerCTA);
|
||||
if (srcShape[d] != resultShape[d]) {
|
||||
assert(srcShape[d] == 1);
|
||||
broadcastDims.push_back(d);
|
||||
srcLogicalShape[d] = 1;
|
||||
srcLogicalShape[d + rank] =
|
||||
std::max<unsigned>(1, triton::gpu::getSizePerThread(srcLayout)[d]);
|
||||
} else {
|
||||
srcLogicalShape[d] = numCtas;
|
||||
srcLogicalShape[d + rank] =
|
||||
triton::gpu::getSizePerThread(resultLayout)[d];
|
||||
}
|
||||
resultLogicalShape[d] = numCtas;
|
||||
resultLogicalShape[d + rank] =
|
||||
triton::gpu::getSizePerThread(resultLayout)[d];
|
||||
|
||||
srcLogicalOrder[d] = order[d] + rank;
|
||||
srcLogicalOrder[d + rank] = order[d];
|
||||
auto srcOffsets = emitOffsetForLayout(srcLayout, srcShape);
|
||||
auto resultOffsets = emitOffsetForLayout(resultLayout, resultShape);
|
||||
SmallVector<Value> srcVals = getElementsFromStruct(loc, src, rewriter);
|
||||
DenseMap<SmallVector<unsigned>, Value, SmallVectorKeyInfo> srcValues;
|
||||
for(size_t i = 0; i < srcOffsets.size(); i++){
|
||||
srcValues[srcOffsets[i]] = srcVals[i];
|
||||
}
|
||||
int64_t duplicates = 1;
|
||||
SmallVector<int64_t> broadcastSizes(broadcastDims.size() * 2);
|
||||
SmallVector<unsigned> broadcastOrder(broadcastDims.size() * 2);
|
||||
for (auto it : llvm::enumerate(broadcastDims)) {
|
||||
// Incase there are multiple indices in the src that is actually
|
||||
// calculating the same element, srcLogicalShape may not need to be 1.
|
||||
// Such as the case when src of shape [256, 1], and with a blocked
|
||||
// layout: sizePerThread: [1, 4]; threadsPerWarp: [1, 32]; warpsPerCTA:
|
||||
// [1, 2]
|
||||
int64_t d = resultLogicalShape[it.value()] / srcLogicalShape[it.value()];
|
||||
broadcastSizes[it.index()] = d;
|
||||
broadcastOrder[it.index()] = srcLogicalOrder[it.value()];
|
||||
duplicates *= d;
|
||||
d = resultLogicalShape[it.value() + rank] /
|
||||
srcLogicalShape[it.value() + rank];
|
||||
broadcastSizes[it.index() + broadcastDims.size()] = d;
|
||||
broadcastOrder[it.index() + broadcastDims.size()] =
|
||||
srcLogicalOrder[it.value() + rank];
|
||||
duplicates *= d;
|
||||
}
|
||||
auto argsort = [](SmallVector<unsigned> input) {
|
||||
SmallVector<unsigned> idx(input.size());
|
||||
std::iota(idx.begin(), idx.end(), 0);
|
||||
std::sort(idx.begin(), idx.end(), [&input](unsigned a, unsigned b) {
|
||||
return input[a] < input[b];
|
||||
});
|
||||
return idx;
|
||||
};
|
||||
broadcastOrder = argsort(broadcastOrder);
|
||||
|
||||
unsigned srcElems = getElemsPerThread(srcTy);
|
||||
auto srcVals = getElementsFromStruct(loc, src, rewriter);
|
||||
unsigned resultElems = getElemsPerThread(resultTy);
|
||||
SmallVector<Value> resultVals(resultElems);
|
||||
for (unsigned i = 0; i < srcElems; ++i) {
|
||||
auto srcMultiDim =
|
||||
getMultiDimIndex<int64_t>(i, srcLogicalShape, srcLogicalOrder);
|
||||
for (int64_t j = 0; j < duplicates; ++j) {
|
||||
auto resultMultiDim = srcMultiDim;
|
||||
auto bcastMultiDim =
|
||||
getMultiDimIndex<int64_t>(j, broadcastSizes, broadcastOrder);
|
||||
for (auto bcastDim : llvm::enumerate(broadcastDims)) {
|
||||
resultMultiDim[bcastDim.value()] += bcastMultiDim[bcastDim.index()];
|
||||
resultMultiDim[bcastDim.value() + rank] +=
|
||||
bcastMultiDim[bcastDim.index() + broadcastDims.size()] *
|
||||
srcLogicalShape[bcastDim.index() + broadcastDims.size()];
|
||||
}
|
||||
auto resultLinearIndex = getLinearIndex<int64_t>(
|
||||
resultMultiDim, resultLogicalShape, srcLogicalOrder);
|
||||
resultVals[resultLinearIndex] = srcVals[i];
|
||||
}
|
||||
SmallVector<Value> resultVals;
|
||||
for(size_t i = 0; i < resultOffsets.size(); i++) {
|
||||
auto offset = resultOffsets[i];
|
||||
for(size_t j = 0; j < srcShape.size(); j++)
|
||||
if(srcShape[j]==1)
|
||||
offset[j] = 0;
|
||||
resultVals.push_back(srcValues.lookup(offset));
|
||||
}
|
||||
auto llvmStructTy = getTypeConverter()->convertType(resultTy);
|
||||
|
||||
Value resultStruct =
|
||||
getStructFromElements(loc, resultVals, rewriter, llvmStructTy);
|
||||
|
||||
rewriter.replaceOp(op, {resultStruct});
|
||||
return success();
|
||||
}
|
||||
@@ -2027,7 +1975,10 @@ struct MakeRangeOpConversion
|
||||
auto idxs = emitIndices(loc, rewriter, layout, shape);
|
||||
unsigned elems = idxs.size();
|
||||
SmallVector<Value> retVals(elems);
|
||||
for (const auto &multiDim : llvm::enumerate(idxs)) {
|
||||
// TODO: slice layout has more elements than expected.
|
||||
// Unexpected behavior for make range, but genereally ok when followed by expand dims + broadcast.
|
||||
// very weird behavior otherwise potentially.
|
||||
for (const auto multiDim : llvm::enumerate(idxs)) {
|
||||
assert(multiDim.value().size() == 1);
|
||||
retVals[multiDim.index()] = add(multiDim.value()[0], start);
|
||||
}
|
||||
@@ -2730,6 +2681,56 @@ public:
|
||||
dstLayout.isa<SliceEncodingAttr>())) {
|
||||
return lowerDistributedToDistributed(op, adaptor, rewriter);
|
||||
}
|
||||
// dot_op<opIdx=0, parent=#mma> = #mma
|
||||
// when #mma = MmaEncoding<version=2, warpsPerCTA=[..., 1]>
|
||||
if(srcLayout.isa<MmaEncodingAttr>() &&
|
||||
dstLayout.isa<DotOperandEncodingAttr>()) {
|
||||
auto srcMmaLayout = srcLayout.cast<MmaEncodingAttr>();
|
||||
auto dstDotLayout = dstLayout.cast<DotOperandEncodingAttr>();
|
||||
if(srcMmaLayout.getWarpsPerCTA()[1] == 1 &&
|
||||
dstDotLayout.getOpIdx() == 0 &&
|
||||
dstDotLayout.getParent() == srcMmaLayout) {
|
||||
// get source values
|
||||
Location loc = op->getLoc();
|
||||
auto vals = getElementsFromStruct(loc, adaptor.src(), rewriter);
|
||||
unsigned elems = getElemsPerThread(srcTy);
|
||||
Type elemTy =
|
||||
this->getTypeConverter()->convertType(srcTy.getElementType());
|
||||
// for the destination type, we need to pack values together
|
||||
// so they can be consumed by tensor core operations
|
||||
unsigned vecSize = std::max<unsigned>(32 / elemTy.getIntOrFloatBitWidth(), 1);
|
||||
Type vecTy = vec_ty(elemTy, vecSize);
|
||||
SmallVector<Type> types(elems/vecSize, vecTy);
|
||||
SmallVector<Value> vecVals;
|
||||
for(unsigned i = 0; i < elems; i += vecSize) {
|
||||
Value packed = rewriter.create<LLVM::UndefOp>(loc, vecTy);
|
||||
for(unsigned j = 0; j < vecSize; j++)
|
||||
packed = insert_element(vecTy, packed, vals[i+j], i32_val(j));
|
||||
vecVals.push_back(packed);
|
||||
}
|
||||
|
||||
// This needs to be ordered the same way that
|
||||
// ldmatrix.x4 would order it
|
||||
// TODO: this needs to be refactor so we don't
|
||||
// implicitly depends on how emitOffsetsForMMAV2
|
||||
// is implemented
|
||||
SmallVector<Value> reorderedVals;
|
||||
for(unsigned i = 0; i < vecVals.size(); i += 4) {
|
||||
reorderedVals.push_back(vecVals[i]);
|
||||
reorderedVals.push_back(vecVals[i+2]);
|
||||
reorderedVals.push_back(vecVals[i+1]);
|
||||
reorderedVals.push_back(vecVals[i+3]);
|
||||
}
|
||||
|
||||
// return composeValuesToDotOperandLayoutStruct(ha, numRepM, numRepK);
|
||||
|
||||
|
||||
Type structTy = LLVM::LLVMStructType::getLiteral(this->getContext(), types);
|
||||
Value view = getStructFromElements(loc, reorderedVals, rewriter, structTy);
|
||||
rewriter.replaceOp(op, view);
|
||||
return success();
|
||||
}
|
||||
}
|
||||
// TODO: to be implemented
|
||||
llvm_unreachable("unsupported layout conversion");
|
||||
return failure();
|
||||
@@ -2853,7 +2854,7 @@ private:
|
||||
emitBaseIndexForBlockedLayout(loc, rewriter, blockedLayout, shape);
|
||||
SmallVector<Value> multiDimOffset(rank);
|
||||
SmallVector<unsigned> multiDimElemId = getMultiDimIndex<unsigned>(
|
||||
elemId, blockedLayout.getSizePerThread(), blockedLayout.getOrder());
|
||||
elemId, getSizePerThread(layout), getOrder(layout));
|
||||
for (unsigned d = 0; d < rank; ++d) {
|
||||
multiDimOffset[d] = add(multiDimOffsetFirstElem[d],
|
||||
idx_val(multiDimCTAInRepId[d] * shapePerCTA[d] +
|
||||
|
@@ -50,10 +50,22 @@ public:
|
||||
auto dstType = convert.getType().cast<RankedTensorType>();
|
||||
if (srcType.getEncoding().isa<triton::gpu::BlockedEncodingAttr>() &&
|
||||
dstType.getEncoding().isa<triton::gpu::DotOperandEncodingAttr>()) {
|
||||
auto dstDotOperand = dstType.getEncoding().cast<triton::gpu::DotOperandEncodingAttr>();
|
||||
auto dstParent = dstDotOperand.getParent();
|
||||
if(dstDotOperand.getOpIdx()==1 ||
|
||||
!dstParent.isa<triton::gpu::MmaEncodingAttr>())
|
||||
return mlir::failure();
|
||||
auto dstParentMma = dstParent.cast<triton::gpu::MmaEncodingAttr>();
|
||||
if(dstParentMma.getVersion() == 1 ||
|
||||
dstParentMma.getWarpsPerCTA()[1] > 1)
|
||||
return mlir::failure();
|
||||
SetVector<Operation*> bwdSlices;
|
||||
mlir::getBackwardSlice(convert.getResult(), &bwdSlices);
|
||||
if(llvm::find_if(bwdSlices, [](Operation *op) { return isa<triton::DotOp>(op); }) == bwdSlices.end())
|
||||
return mlir::failure();
|
||||
|
||||
auto tmpType =
|
||||
RankedTensorType::get(dstType.getShape(), dstType.getElementType(),
|
||||
triton::gpu::SharedEncodingAttr::get(
|
||||
op->getContext(), 1, 1, 1, {1, 0}));
|
||||
RankedTensorType::get(dstType.getShape(), dstType.getElementType(), dstParentMma);
|
||||
auto tmp = rewriter.create<triton::gpu::ConvertLayoutOp>(
|
||||
convert.getLoc(), tmpType, convert.getOperand());
|
||||
auto newConvert = rewriter.create<triton::gpu::ConvertLayoutOp>(
|
||||
@@ -81,8 +93,11 @@ public:
|
||||
auto convert = llvm::cast<triton::gpu::ConvertLayoutOp>(op);
|
||||
// we don't handle conversions to DotOperandEncodingAttr
|
||||
// this is a heuristics to accommodate fused attention
|
||||
// if (dstType.getEncoding().isa<triton::gpu::DotOperandEncodingAttr>())
|
||||
// return mlir::failure();
|
||||
auto srcType = convert.getOperand().getType().cast<RankedTensorType>();
|
||||
auto dstType = convert.getType().cast<RankedTensorType>();
|
||||
if (dstType.getEncoding().isa<triton::gpu::DotOperandEncodingAttr>() &&
|
||||
srcType.getEncoding().isa<triton::gpu::MmaEncodingAttr>())
|
||||
return mlir::failure();
|
||||
// convert to the same layout -- we can delete
|
||||
if (op->getResultTypes() == op->getOperandTypes()) {
|
||||
rewriter.replaceOp(op, op->getOperands());
|
||||
@@ -586,12 +601,9 @@ mmaVersionToShapePerWarp(int version, const ArrayRef<int64_t> &shape,
|
||||
}
|
||||
}
|
||||
|
||||
template <int version>
|
||||
SmallVector<unsigned, 2> warpsPerTile(const ArrayRef<int64_t> shape,
|
||||
int numWarps);
|
||||
|
||||
template <>
|
||||
SmallVector<unsigned, 2> warpsPerTile<1>(const ArrayRef<int64_t> shape,
|
||||
SmallVector<unsigned, 2> warpsPerTileV1(triton::DotOp dotOp,
|
||||
const ArrayRef<int64_t> shape,
|
||||
int numWarps) {
|
||||
SmallVector<unsigned, 2> ret = {1, 1};
|
||||
SmallVector<int64_t, 2> shapePerWarp =
|
||||
@@ -611,33 +623,40 @@ SmallVector<unsigned, 2> warpsPerTile<1>(const ArrayRef<int64_t> shape,
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
SmallVector<unsigned, 2> warpsPerTile<2>(const ArrayRef<int64_t> shape,
|
||||
SmallVector<unsigned, 2> warpsPerTileV2(triton::DotOp dotOp,
|
||||
const ArrayRef<int64_t> shape,
|
||||
int numWarps) {
|
||||
SmallVector<unsigned, 2> ret = {1, 1};
|
||||
SmallVector<int64_t, 2> shapePerWarp =
|
||||
mmaVersionToShapePerWarp(2, shape, numWarps);
|
||||
// TODO (@daadaada): double-check.
|
||||
// original logic in
|
||||
// https://github.com/openai/triton/blob/master/lib/codegen/analysis/layout.cc#L252
|
||||
// seems buggy for shape = [32, 16] ?
|
||||
do {
|
||||
if (ret[0] * ret[1] >= numWarps)
|
||||
break;
|
||||
if (shape[0] / shapePerWarp[0] / ret[0] >=
|
||||
shape[1] / (shapePerWarp[1] * 2) / ret[1]) {
|
||||
if (ret[0] < shape[0] / shapePerWarp[0]) {
|
||||
ret[0] *= 2;
|
||||
} else
|
||||
SetVector<Operation*> slices;
|
||||
mlir::getForwardSlice(dotOp.getResult(), &slices);
|
||||
if(llvm::find_if(slices, [](Operation *op) { return isa<triton::DotOp>(op); }) != slices.end())
|
||||
return {(unsigned)numWarps, 1};
|
||||
|
||||
SmallVector<unsigned, 2> ret = {1, 1};
|
||||
SmallVector<int64_t, 2> shapePerWarp = {16, 8};
|
||||
bool changed = false;
|
||||
// TODO (@daadaada): double-check.
|
||||
// original logic in
|
||||
// https://github.com/openai/triton/blob/master/lib/codegen/analysis/layout.cc#L252
|
||||
// seems buggy for shape = [32, 16] ?
|
||||
do {
|
||||
changed = false;
|
||||
if (ret[0] * ret[1] >= numWarps)
|
||||
break;
|
||||
if (shape[0] / shapePerWarp[0] / ret[0] >=
|
||||
shape[1] / (shapePerWarp[1] * 2) / ret[1]) {
|
||||
if (ret[0] < shape[0] / shapePerWarp[0]) {
|
||||
ret[0] *= 2;
|
||||
} else
|
||||
ret[1] *= 2;
|
||||
} else {
|
||||
ret[1] *= 2;
|
||||
} else {
|
||||
ret[1] *= 2;
|
||||
}
|
||||
} while (true);
|
||||
return ret;
|
||||
}
|
||||
} while (true);
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
class BlockedToMMA : public mlir::RewritePattern {
|
||||
int computeCapability;
|
||||
|
||||
@@ -646,13 +665,14 @@ public:
|
||||
: mlir::RewritePattern(triton::DotOp::getOperationName(), 2, context),
|
||||
computeCapability(computeCapability) {}
|
||||
|
||||
static SmallVector<unsigned, 2> getWarpsPerTile(const ArrayRef<int64_t> shape,
|
||||
static SmallVector<unsigned, 2> getWarpsPerTile(triton::DotOp dotOp,
|
||||
const ArrayRef<int64_t> shape,
|
||||
int version, int numWarps) {
|
||||
switch (version) {
|
||||
case 1:
|
||||
return warpsPerTile<1>(shape, numWarps);
|
||||
return warpsPerTileV1(dotOp, shape, numWarps);
|
||||
case 2:
|
||||
return warpsPerTile<2>(shape, numWarps);
|
||||
return warpsPerTileV2(dotOp, shape, numWarps);
|
||||
default:
|
||||
assert(false && "not supported version");
|
||||
return {0, 0};
|
||||
@@ -684,7 +704,7 @@ public:
|
||||
retShape, oldRetType.getElementType(),
|
||||
triton::gpu::MmaEncodingAttr::get(
|
||||
oldRetType.getContext(), version,
|
||||
getWarpsPerTile(retShape, version, numWarps)));
|
||||
getWarpsPerTile(dotOp, retShape, version, numWarps)));
|
||||
// convert accumulator
|
||||
auto oldAcc = dotOp.getOperand(2);
|
||||
auto newAcc = rewriter.create<triton::gpu::ConvertLayoutOp>(
|
||||
@@ -732,7 +752,7 @@ public:
|
||||
mlir::RewritePatternSet patterns(context);
|
||||
|
||||
patterns.add<SimplifyConversion>(context);
|
||||
// patterns.add<DecomposeDotOperand>(context);
|
||||
patterns.add<DecomposeDotOperand>(context);
|
||||
patterns.add<RematerializeBackward>(context);
|
||||
patterns.add<RematerializeForward>(context);
|
||||
patterns.add<MoveConvertOutOfLoop>(context);
|
||||
|
@@ -130,6 +130,11 @@ LogicalResult Prefetcher::initialize() {
|
||||
|
||||
if (dotsInFor.empty())
|
||||
return failure();
|
||||
|
||||
// TODO: segfault (original for still has uses)
|
||||
// when used in flash attention that has 2 dots in the loop
|
||||
if(dotsInFor.size() > 1)
|
||||
return failure();
|
||||
|
||||
// returns source of cvt
|
||||
auto getPrefetchSrc = [](Value v) -> Value {
|
||||
|
Reference in New Issue
Block a user