2020-07-07 04:57:00 +08:00
|
|
|
/* Copyright 2019 The TensorFlow Authors. All Rights Reserved.
|
|
|
|
|
|
|
|
Licensed under the Apache License, Version 2.0 (the "License");
|
|
|
|
you may not use this file except in compliance with the License.
|
|
|
|
You may obtain a copy of the License at
|
|
|
|
|
|
|
|
http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
|
|
|
|
Unless required by applicable law or agreed to in writing, software
|
|
|
|
distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
See the License for the specific language governing permissions and
|
|
|
|
limitations under the License.
|
|
|
|
==============================================================================*/
|
|
|
|
|
|
|
|
// This file implements logic for lowering HLO/LHLO dialect to Linalg dialect.
|
|
|
|
|
2020-08-22 14:26:35 +08:00
|
|
|
#include <numeric>
|
|
|
|
|
2020-09-23 00:06:55 +08:00
|
|
|
#include "llvm/ADT/STLExtras.h"
|
2021-01-28 21:44:49 +08:00
|
|
|
#include "llvm/ADT/SetVector.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir-hlo/Dialect/mhlo/IR/hlo_ops.h"
|
|
|
|
#include "mlir-hlo/Dialect/mhlo/IR/lhlo_ops.h"
|
|
|
|
#include "mlir-hlo/Dialect/mhlo/transforms/map_lmhlo_to_scalar_op.h"
|
|
|
|
#include "mlir-hlo/Dialect/mhlo/transforms/rewriters.h"
|
|
|
|
#include "mlir/Dialect/Affine/IR/AffineOps.h"
|
|
|
|
#include "mlir/Dialect/Linalg/IR/LinalgOps.h"
|
|
|
|
#include "mlir/Dialect/Linalg/IR/LinalgTypes.h"
|
2021-02-13 00:30:51 +08:00
|
|
|
#include "mlir/Dialect/Math/IR/Math.h"
|
2021-03-30 16:06:12 +08:00
|
|
|
#include "mlir/Dialect/MemRef/IR/MemRef.h"
|
2020-12-22 07:26:38 +08:00
|
|
|
#include "mlir/Dialect/SCF/SCF.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/Dialect/StandardOps/IR/Ops.h"
|
2021-01-12 02:11:39 +08:00
|
|
|
#include "mlir/Dialect/Tensor/IR/Tensor.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/IR/AffineExpr.h"
|
|
|
|
#include "mlir/IR/Attributes.h"
|
|
|
|
#include "mlir/IR/Builders.h"
|
2020-12-02 05:17:12 +08:00
|
|
|
#include "mlir/IR/BuiltinOps.h"
|
2020-12-15 16:58:42 +08:00
|
|
|
#include "mlir/IR/BuiltinTypes.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/IR/Location.h"
|
|
|
|
#include "mlir/IR/MLIRContext.h"
|
2021-01-28 21:44:49 +08:00
|
|
|
#include "mlir/IR/Matchers.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/IR/Operation.h"
|
2020-09-23 00:06:55 +08:00
|
|
|
#include "mlir/IR/OperationSupport.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/IR/PatternMatch.h"
|
2020-09-23 00:06:55 +08:00
|
|
|
#include "mlir/IR/TypeUtilities.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/Pass/Pass.h"
|
2021-01-28 21:44:49 +08:00
|
|
|
#include "mlir/Pass/PassManager.h"
|
2021-02-25 01:43:54 +08:00
|
|
|
#include "mlir/Support/LogicalResult.h"
|
2020-07-29 07:12:08 +08:00
|
|
|
#include "mlir/Transforms/DialectConversion.h"
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
namespace mlir {
|
|
|
|
namespace {
|
|
|
|
|
2021-01-28 21:44:49 +08:00
|
|
|
/// Returns an ArrayAttr that contains `nLoops` attributes. All the attributes
|
|
|
|
/// are "parallel" except the last `nReduction` elements, where are "reduction"
|
|
|
|
/// attributes.
|
|
|
|
SmallVector<StringRef, 3> GetParallelAndReductionIterators(
|
|
|
|
unsigned nLoops, unsigned nReduction) {
|
|
|
|
SmallVector<StringRef, 3> res(nLoops - nReduction,
|
|
|
|
getParallelIteratorTypeName());
|
|
|
|
res.append(nReduction, getReductionIteratorTypeName());
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
SmallVector<StringRef, 3> GetNParallelLoopsAttrs(unsigned nParallelLoops) {
|
2021-01-28 21:44:49 +08:00
|
|
|
return GetParallelAndReductionIterators(nParallelLoops, 0);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
template <bool isLHLO = true>
|
2020-12-15 02:46:04 +08:00
|
|
|
Value GetResultValue(Operation* op) {
|
2020-07-07 04:57:00 +08:00
|
|
|
return isLHLO ? op->getOperand(op->getNumOperands() - 1) : op->getResult(0);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <bool isLHLO = true>
|
2020-12-15 02:46:04 +08:00
|
|
|
ShapedType GetHloOpResultType(Operation* op) {
|
|
|
|
return GetResultValue<isLHLO>(op).getType().template cast<ShapedType>();
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
template <bool isLHLO = true>
|
2020-12-15 02:46:04 +08:00
|
|
|
bool VerifyHloOpBufferOrTensorSemantics(Operation* op) {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto verify_type = [&](Value val) -> bool {
|
2020-07-07 04:57:00 +08:00
|
|
|
return (isLHLO && val.getType().isa<MemRefType>()) ||
|
|
|
|
(!isLHLO && val.getType().isa<RankedTensorType>());
|
|
|
|
};
|
2020-10-24 03:22:21 +08:00
|
|
|
if (!llvm::all_of(op->getOperands(), verify_type)) return false;
|
2020-07-07 04:57:00 +08:00
|
|
|
return isLHLO ? op->getResults().empty()
|
2020-10-24 03:22:21 +08:00
|
|
|
: llvm::all_of(op->getResults(), verify_type);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
|
2020-12-24 15:53:08 +08:00
|
|
|
Value GetInitTensor(OpBuilder& b, Location loc, ShapedType type,
|
2021-02-04 07:02:21 +08:00
|
|
|
ArrayRef<Value> dyn_sizes) {
|
2020-12-24 15:53:08 +08:00
|
|
|
return b.create<linalg::InitTensorOp>(loc, dyn_sizes, type.getShape(),
|
|
|
|
type.getElementType());
|
|
|
|
}
|
|
|
|
|
|
|
|
SmallVector<Value, 2> ExtractDynamicSizes(OpBuilder& b, Location loc,
|
2021-03-12 00:30:08 +08:00
|
|
|
Value tensor,
|
|
|
|
Value shape_tensor = nullptr) {
|
2020-12-24 15:53:08 +08:00
|
|
|
auto tensor_type = tensor.getType().dyn_cast<RankedTensorType>();
|
|
|
|
if (!tensor_type) return {};
|
|
|
|
SmallVector<Value, 2> dyn_sizes;
|
|
|
|
for (auto& en : llvm::enumerate(tensor_type.getShape())) {
|
|
|
|
if (en.value() != ShapedType::kDynamicSize) continue;
|
2021-03-12 00:30:08 +08:00
|
|
|
// If a shape tensor is present extract from there.
|
|
|
|
if (shape_tensor) {
|
|
|
|
Value extract = b.create<tensor::ExtractOp>(
|
|
|
|
loc, shape_tensor,
|
|
|
|
ValueRange{b.create<ConstantIndexOp>(loc, en.index())});
|
|
|
|
dyn_sizes.push_back(
|
|
|
|
b.create<IndexCastOp>(loc, b.getIndexType(), extract));
|
|
|
|
} else {
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_sizes.push_back(b.create<memref::DimOp>(loc, tensor, en.index()));
|
2021-03-12 00:30:08 +08:00
|
|
|
}
|
2020-12-24 15:53:08 +08:00
|
|
|
}
|
|
|
|
return dyn_sizes;
|
|
|
|
}
|
|
|
|
|
2021-01-13 14:07:29 +08:00
|
|
|
SmallVector<int64_t, 4> Extract1DVector(DenseIntElementsAttr elements) {
|
|
|
|
SmallVector<int64_t, 4> ret;
|
|
|
|
for (const APInt& element : elements) {
|
|
|
|
ret.push_back(element.getLimitedValue());
|
|
|
|
}
|
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
2021-01-28 21:44:49 +08:00
|
|
|
/// Returns the constant value associated with the init value if the defining
|
|
|
|
/// operation is a constant.
|
|
|
|
Attribute GetInitValueAsConst(Value init) {
|
|
|
|
DenseElementsAttr attr;
|
|
|
|
if (!matchPattern(init, m_Constant(&attr))) return {};
|
|
|
|
auto type = attr.getType().dyn_cast<ShapedType>();
|
|
|
|
if (!type || type.getRank() != 0) return {};
|
|
|
|
return attr.getValue({});
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Returns a permutation AffineMap that puts all reduction dimensions to the
|
|
|
|
/// last. The order of parallel loops and reduction loops are all sorted. E.g.,
|
|
|
|
/// if `rank` is 4 and `reductionDims` is {1, 3}, then
|
|
|
|
/// "(d0, d1, d2, d3) -> (d0, d2, d1, d3)" is used. The inverse permutation of
|
|
|
|
/// the AffineMap is returned.
|
|
|
|
AffineMap GetTransposeMapForReduction(MLIRContext* context, int rank,
|
|
|
|
ArrayRef<int64_t> reduction_dims) {
|
|
|
|
llvm::SmallSetVector<int, 4> s;
|
|
|
|
for (auto dim : reduction_dims) s.insert(dim);
|
|
|
|
|
|
|
|
SmallVector<unsigned, 4> permutation;
|
|
|
|
for (int i = 0; i < rank; ++i)
|
|
|
|
if (!s.count(i)) permutation.push_back(i);
|
|
|
|
for (auto dim : reduction_dims) permutation.push_back(dim);
|
|
|
|
|
|
|
|
auto map = AffineMap::getPermutationMap(permutation, context);
|
|
|
|
return inversePermutation(map);
|
|
|
|
}
|
|
|
|
|
2021-03-12 00:40:31 +08:00
|
|
|
/// Returns true if the given `attr` is a splat of the given `value`.
|
|
|
|
bool isSplatValue(DenseIntElementsAttr attr, uint64_t value) {
|
|
|
|
return attr.isSplat() && attr.getSplatValue<uint64_t>() == value;
|
|
|
|
}
|
|
|
|
|
2021-02-25 21:58:12 +08:00
|
|
|
/// Returns true if the given `dimensionNumbers` from a mhlo.convolution op
|
|
|
|
/// follows a canonical form:
|
|
|
|
///
|
|
|
|
/// * Input dimensions have order: (batch_count, spatial_dims,
|
|
|
|
/// input_channel_count).
|
|
|
|
/// * Filter dimensions have order: (spatial_dims, input_channel_count,
|
|
|
|
/// output_channel_count).
|
|
|
|
/// * Output dimensions have order: (batch_count, spatial_dims,
|
|
|
|
/// output_channel_count).
|
|
|
|
template <typename DimensionNumbersTy>
|
|
|
|
static bool HasCanonicalDimensionNumbers(
|
|
|
|
const DimensionNumbersTy& dimension_numbers) {
|
|
|
|
const int input_spatial_rank =
|
|
|
|
llvm::size(dimension_numbers.input_spatial_dimensions());
|
|
|
|
// The dimensions for input should follow the order of
|
|
|
|
// batch_count, spatial_dims..., input_feature_count.
|
|
|
|
if (dimension_numbers.input_batch_dimension().getInt() != 0 ||
|
|
|
|
dimension_numbers.input_feature_dimension().getInt() !=
|
|
|
|
(input_spatial_rank + 1)) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
const int kernel_spatial_rank =
|
|
|
|
llvm::size(dimension_numbers.kernel_spatial_dimensions());
|
|
|
|
// The dimensions for filter should follow the order of
|
|
|
|
// spatial_dims..., input_feature_count, num_output_feature_count.
|
|
|
|
if (dimension_numbers.kernel_input_feature_dimension().getInt() !=
|
|
|
|
kernel_spatial_rank ||
|
|
|
|
dimension_numbers.kernel_output_feature_dimension().getInt() !=
|
|
|
|
(kernel_spatial_rank + 1)) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
const int output_spatial_rank =
|
|
|
|
llvm::size(dimension_numbers.output_spatial_dimensions());
|
|
|
|
// The dimensions for output should follow the order of
|
|
|
|
// batch_count, spatial_dims.., output_feature_count.
|
|
|
|
if (dimension_numbers.output_batch_dimension().getInt() != 0 ||
|
|
|
|
dimension_numbers.output_feature_dimension().getInt() !=
|
|
|
|
(output_spatial_rank + 1)) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (input_spatial_rank != output_spatial_rank ||
|
|
|
|
input_spatial_rank != kernel_spatial_rank) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
auto input_spatial_dim = dimension_numbers.input_spatial_dimensions().begin();
|
|
|
|
auto kernel_spatial_dim =
|
|
|
|
dimension_numbers.kernel_spatial_dimensions().begin();
|
|
|
|
auto output_spatial_dim =
|
|
|
|
dimension_numbers.output_spatial_dimensions().begin();
|
|
|
|
// Check spatial dims are ordered correctly.
|
|
|
|
for (int i = 0; i < input_spatial_rank; ++i) {
|
|
|
|
const int dim = i + 1;
|
|
|
|
if ((*input_spatial_dim++).getZExtValue() != dim ||
|
|
|
|
(*output_spatial_dim++).getZExtValue() != dim ||
|
|
|
|
(*kernel_spatial_dim++).getZExtValue() != i) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class PointwiseToLinalgConverter : public OpConversionPattern<OpTy> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
OpTy op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
auto loc = op.getLoc();
|
2020-09-23 00:06:55 +08:00
|
|
|
ShapedType t0 = args[0].getType().template dyn_cast<ShapedType>();
|
|
|
|
if (!t0) return failure();
|
|
|
|
|
|
|
|
unsigned nloops = t0.getRank();
|
|
|
|
auto fail = [&](ShapedType t) {
|
|
|
|
return !t || !t.hasRank() || t.getRank() != nloops ||
|
|
|
|
!(t.getElementType().isSignlessIntOrFloat() ||
|
|
|
|
t.getElementType().isa<ComplexType>());
|
|
|
|
};
|
2021-04-29 17:26:25 +08:00
|
|
|
if (llvm::any_of(op.getOperation()->getResultTypes(), [&](Type t) {
|
|
|
|
return fail(this->typeConverter->convertType(t)
|
|
|
|
.template dyn_cast<ShapedType>());
|
|
|
|
})) {
|
2020-09-23 00:06:55 +08:00
|
|
|
return emitError(loc,
|
2021-04-29 17:26:25 +08:00
|
|
|
"hlo to linalg conversion expects ranked args of "
|
2020-09-23 00:06:55 +08:00
|
|
|
"signless int, float or complex element type with ")
|
|
|
|
<< nloops << " parallel iterators: " << *(op.getOperation());
|
2021-04-29 17:26:25 +08:00
|
|
|
}
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// Construct the indexing maps needed for linalg.generic ops.
|
2020-10-24 03:22:21 +08:00
|
|
|
SmallVector<Type, 4> body_arg_types, body_result_types, op_result_types;
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// This doesnt account for implicit broadcast, but the working assumption
|
2020-09-23 00:06:55 +08:00
|
|
|
// in HLO/LHLO is that are broadcasts are made explicit.
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
if (isLHLO && !nloops) return failure();
|
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
int num_inputs = (isLHLO ? args.size() - 1 : args.size());
|
2020-09-23 00:06:55 +08:00
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
ValueRange inputs(args.take_front(num_inputs));
|
2020-09-23 00:06:55 +08:00
|
|
|
for (Value in : inputs)
|
2020-10-24 03:22:21 +08:00
|
|
|
body_arg_types.emplace_back(getElementTypeOrSelf(in.getType()));
|
2020-09-23 00:06:55 +08:00
|
|
|
|
2020-12-24 15:53:08 +08:00
|
|
|
SmallVector<Value, 4> output_buffers;
|
|
|
|
if (isLHLO) {
|
|
|
|
output_buffers.append(args.begin() + num_inputs, args.end());
|
|
|
|
} else {
|
2021-04-29 17:26:25 +08:00
|
|
|
Type result_type = this->typeConverter->convertType(
|
|
|
|
op.getOperation()->getResult(0).getType());
|
2020-12-24 15:53:08 +08:00
|
|
|
auto dyn_sizes = ExtractDynamicSizes(rewriter, loc, args[0]);
|
2021-04-29 17:26:25 +08:00
|
|
|
output_buffers.push_back(GetInitTensor(
|
|
|
|
rewriter, loc, result_type.cast<ShapedType>(), dyn_sizes));
|
|
|
|
op_result_types.push_back(result_type);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2020-12-24 15:53:08 +08:00
|
|
|
body_result_types = llvm::to_vector<4>(llvm::map_range(
|
|
|
|
output_buffers, [](Value v) { return getElementTypeOrSelf(v); }));
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
AffineMap common_indexing_map =
|
2020-09-23 00:06:55 +08:00
|
|
|
nloops ? rewriter.getMultiDimIdentityMap(nloops)
|
|
|
|
: AffineMap::get(nloops, 0, rewriter.getContext());
|
|
|
|
SmallVector<AffineMap, 2> indexing_maps(args.size() + (isLHLO ? 0 : 1),
|
2020-10-24 03:22:21 +08:00
|
|
|
common_indexing_map);
|
2020-09-23 00:06:55 +08:00
|
|
|
|
2020-12-16 20:57:35 +08:00
|
|
|
bool failed = false;
|
2020-10-24 03:22:21 +08:00
|
|
|
auto linalg_op = rewriter.create<linalg::GenericOp>(
|
2020-12-24 15:53:08 +08:00
|
|
|
loc, op_result_types, inputs, output_buffers, indexing_maps,
|
2020-07-07 04:57:00 +08:00
|
|
|
GetNParallelLoopsAttrs(nloops),
|
2020-10-24 03:22:21 +08:00
|
|
|
[&](OpBuilder& nested_builder, Location nested_loc, ValueRange args) {
|
2020-07-09 01:05:32 +08:00
|
|
|
// TODO(ravishankarm) : For now use the method in lmhlo namespace.
|
2020-07-07 04:57:00 +08:00
|
|
|
// That method needs to be moved out of there.
|
2020-10-24 03:22:21 +08:00
|
|
|
Value op_result = lmhlo::HloOpToStdScalarOp::map<OpTy>(
|
|
|
|
op, body_result_types,
|
2020-09-23 00:06:55 +08:00
|
|
|
llvm::to_vector<2>(args.take_front(inputs.size())), &rewriter);
|
2020-12-16 20:57:35 +08:00
|
|
|
if (op_result == nullptr) {
|
|
|
|
failed = true;
|
|
|
|
} else {
|
|
|
|
nested_builder.create<linalg::YieldOp>(loc, op_result);
|
|
|
|
}
|
2020-07-07 04:57:00 +08:00
|
|
|
});
|
2020-12-16 20:57:35 +08:00
|
|
|
if (failed) return failure();
|
2020-10-24 03:22:21 +08:00
|
|
|
rewriter.replaceOp(op, linalg_op.getOperation()->getResults());
|
2020-07-07 04:57:00 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename LhloOp>
|
|
|
|
class ScalarPointwiseToStandardConverter : public OpConversionPattern<LhloOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<LhloOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
LhloOp lhlo_op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
auto loc = lhlo_op.getLoc();
|
2020-10-24 03:22:21 +08:00
|
|
|
auto arg_type =
|
2020-07-07 04:57:00 +08:00
|
|
|
lhlo_op.getOperand(0).getType().template dyn_cast<ShapedType>();
|
2020-10-24 03:22:21 +08:00
|
|
|
if (!arg_type || !arg_type.getElementType().isSignlessIntOrFloat() ||
|
|
|
|
(arg_type.getRank() != 0)) {
|
2020-07-07 04:57:00 +08:00
|
|
|
return failure();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create two loads from the input.
|
2021-03-17 04:31:59 +08:00
|
|
|
auto lhs = rewriter.create<memref::LoadOp>(loc, lhlo_op.lhs());
|
|
|
|
auto rhs = rewriter.create<memref::LoadOp>(loc, lhlo_op.rhs());
|
2020-07-09 01:05:32 +08:00
|
|
|
// TODO(ravishankarm) : Move this method out of lmhlo namespace.
|
2020-10-24 03:22:21 +08:00
|
|
|
Value op_result = lmhlo::HloOpToStdScalarOp::map<LhloOp>(
|
|
|
|
lhlo_op, arg_type.getElementType(), llvm::ArrayRef<Value>{lhs, rhs},
|
2020-07-07 04:57:00 +08:00
|
|
|
&rewriter);
|
2021-03-17 04:31:59 +08:00
|
|
|
rewriter.create<memref::StoreOp>(loc, op_result, lhlo_op.out());
|
2020-07-07 04:57:00 +08:00
|
|
|
rewriter.eraseOp(lhlo_op);
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
//===----------------------------------------------------------------------===//
|
2020-07-09 01:05:32 +08:00
|
|
|
// lmhlo.convolution conversion pattern.
|
2020-07-07 04:57:00 +08:00
|
|
|
//===----------------------------------------------------------------------===//
|
|
|
|
|
2020-07-09 01:05:32 +08:00
|
|
|
/// Converts lmhlo.convolution operation to a linalg.conv op.
|
|
|
|
struct ConvToLinalgConverter : public OpConversionPattern<lmhlo::ConvOp> {
|
2020-07-07 04:57:00 +08:00
|
|
|
public:
|
2020-07-09 01:05:32 +08:00
|
|
|
using OpConversionPattern<lmhlo::ConvOp>::OpConversionPattern;
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
2020-07-09 01:05:32 +08:00
|
|
|
lmhlo::ConvOp op, ArrayRef<Value> args,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2021-02-25 21:58:12 +08:00
|
|
|
if (!HasCanonicalDimensionNumbers(op.dimension_numbers())) return failure();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// TODO: LHS dilation for deconvolution not supported yet.
|
2020-12-11 08:38:26 +08:00
|
|
|
// TODO(jurahul): Window reversal is not supported yet.
|
|
|
|
if (op.lhs_dilation() || op.hasWindowReversal()) {
|
2020-07-07 04:57:00 +08:00
|
|
|
return failure();
|
|
|
|
}
|
|
|
|
|
|
|
|
llvm::SmallVector<Attribute, 4> strides;
|
2020-10-24 03:22:21 +08:00
|
|
|
if (auto window_strides = op.window_strides()) {
|
|
|
|
auto range = window_strides->getAttributeValues();
|
2020-07-07 04:57:00 +08:00
|
|
|
strides.assign(range.begin(), range.end());
|
|
|
|
}
|
2021-02-09 00:01:22 +08:00
|
|
|
auto strides_arg = ArrayAttr::get(op.getContext(), strides);
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
llvm::SmallVector<Attribute, 2> dilation;
|
2020-10-24 03:22:21 +08:00
|
|
|
if (auto rhs_dilation = op.rhs_dilation()) {
|
|
|
|
auto range = rhs_dilation->getAttributeValues();
|
2020-07-07 04:57:00 +08:00
|
|
|
dilation.assign(range.begin(), range.end());
|
|
|
|
} else {
|
|
|
|
// Default dilation of 1.
|
|
|
|
dilation.resize(2, IntegerAttr::get(rewriter.getIntegerType(64), 1));
|
|
|
|
}
|
2021-02-09 00:01:22 +08:00
|
|
|
auto dilation_arg = ArrayAttr::get(op.getContext(), dilation);
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// Set padding only if it is non-zero.
|
|
|
|
DenseIntElementsAttr padding = op.paddingAttr();
|
2020-10-24 03:22:21 +08:00
|
|
|
if (!padding ||
|
|
|
|
!llvm::any_of(padding.getValues<APInt>(),
|
|
|
|
[](APInt int_val) { return !int_val.isNullValue(); })) {
|
2020-07-07 04:57:00 +08:00
|
|
|
padding = nullptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
// The order of input and filter are switched with linalg.conv.
|
|
|
|
rewriter.replaceOpWithNewOp<linalg::ConvOp>(
|
2020-10-24 03:22:21 +08:00
|
|
|
op, args[1], args[0], args[2], strides_arg, dilation_arg, padding);
|
2020-07-07 04:57:00 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-09 11:32:16 +08:00
|
|
|
/// Base class for lowering HLO operations that have one operand and one result,
|
2020-07-07 04:57:00 +08:00
|
|
|
/// and are semantically equivalent to a copy of the input to the output (like
|
|
|
|
/// transpose, some reshape, etc.). The derived classes need to provide a method
|
|
|
|
/// `getIndexingMaps` that returns AffineMaps for the index maps of the input
|
|
|
|
/// and the output.
|
|
|
|
template <typename Derived, typename OpTy, bool isLHLO = true>
|
|
|
|
class DataMovementOpConverter : public OpConversionPattern<OpTy> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
OpTy op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2020-12-15 02:46:04 +08:00
|
|
|
if (!VerifyHloOpBufferOrTensorSemantics<isLHLO>(op)) return failure();
|
|
|
|
auto result_type = GetHloOpResultType<isLHLO>(op);
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
SmallVector<AffineMap, 2> indexing_maps =
|
|
|
|
Derived::getIndexingMaps(op, &rewriter);
|
|
|
|
if (indexing_maps.empty()) return failure();
|
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
auto nloops = result_type.getRank();
|
2020-07-07 04:57:00 +08:00
|
|
|
auto loc = op.getLoc();
|
2020-12-24 15:53:08 +08:00
|
|
|
// TODO(pifon): technically, the op itself could have size operands (e.g.
|
|
|
|
// broadcast into a dynamic dimension).Handle this case.
|
|
|
|
auto dyn_sizes = isLHLO ? SmallVector<Value, 2>()
|
|
|
|
: ExtractDynamicSizes(rewriter, loc, args[0]);
|
2020-10-24 03:22:21 +08:00
|
|
|
auto linalg_op = rewriter.create<linalg::GenericOp>(
|
2020-09-23 00:06:55 +08:00
|
|
|
loc,
|
2020-10-24 03:22:21 +08:00
|
|
|
/*resultTensorTypes=*/isLHLO ? ArrayRef<Type>{} : result_type,
|
2020-09-23 00:06:55 +08:00
|
|
|
/*inputs=*/args.front(),
|
2020-12-24 15:53:08 +08:00
|
|
|
/*outputBuffers=*/
|
2021-02-04 07:02:21 +08:00
|
|
|
isLHLO
|
|
|
|
? ValueRange{args.back()}
|
|
|
|
: ValueRange{GetInitTensor(rewriter, loc, result_type, dyn_sizes)},
|
2020-12-24 15:53:08 +08:00
|
|
|
indexing_maps, GetNParallelLoopsAttrs(nloops),
|
2020-10-24 03:22:21 +08:00
|
|
|
[&](OpBuilder& nested_builder, Location nested_loc, ValueRange args) {
|
|
|
|
nested_builder.create<linalg::YieldOp>(loc, *args.begin());
|
2020-07-07 04:57:00 +08:00
|
|
|
});
|
2020-10-24 03:22:21 +08:00
|
|
|
rewriter.replaceOp(op, linalg_op.getOperation()->getResults());
|
2020-07-07 04:57:00 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
/// Pattern to convert BroadcastOp to Linalg ops.
|
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class BroadcastConverter
|
|
|
|
: public DataMovementOpConverter<BroadcastConverter<OpTy, isLHLO>, OpTy,
|
|
|
|
isLHLO> {
|
|
|
|
public:
|
|
|
|
using DataMovementOpConverter<BroadcastConverter, OpTy,
|
|
|
|
isLHLO>::DataMovementOpConverter;
|
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
static SmallVector<AffineMap, 2> getIndexingMaps(OpTy broadcast_op,
|
2020-07-07 04:57:00 +08:00
|
|
|
Builder* b) {
|
2020-10-24 03:22:21 +08:00
|
|
|
ShapedType input_type =
|
|
|
|
broadcast_op.operand().getType().template cast<ShapedType>();
|
|
|
|
unsigned input_rank = input_type.getRank();
|
2020-12-15 02:46:04 +08:00
|
|
|
unsigned nloops = GetHloOpResultType<isLHLO>(broadcast_op).getRank();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// BroadcastOp prepends the dimensions in the `broadcast_sizes` attribute to
|
|
|
|
// the input's dimensions.
|
2020-10-24 03:22:21 +08:00
|
|
|
unsigned num_prepended_dims = llvm::size(broadcast_op.broadcast_sizes());
|
|
|
|
SmallVector<AffineExpr, 4> input_dim_exprs;
|
|
|
|
input_dim_exprs.reserve(input_rank);
|
2021-03-04 23:35:15 +08:00
|
|
|
for (unsigned i = 0; i < input_rank; ++i) {
|
2020-10-24 03:22:21 +08:00
|
|
|
input_dim_exprs.push_back(b->getAffineDimExpr(num_prepended_dims + i));
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
AffineMap input_map;
|
2020-07-07 04:57:00 +08:00
|
|
|
MLIRContext* context = b->getContext();
|
2020-10-24 03:22:21 +08:00
|
|
|
if (input_dim_exprs.empty()) {
|
2020-07-07 04:57:00 +08:00
|
|
|
// The input is a scalar, i.e. this is a scalar broadcast op.
|
2020-10-24 03:22:21 +08:00
|
|
|
input_map = AffineMap::get(nloops, /*symbolCount=*/0, context);
|
2020-07-07 04:57:00 +08:00
|
|
|
} else {
|
2020-10-24 03:22:21 +08:00
|
|
|
input_map =
|
|
|
|
AffineMap::get(nloops, /*symbolCount=*/0, input_dim_exprs, context);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2020-10-24 03:22:21 +08:00
|
|
|
return {input_map, b->getMultiDimIdentityMap(nloops)};
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
class HloBroadcastInDimConverter
|
|
|
|
: public DataMovementOpConverter<HloBroadcastInDimConverter,
|
2020-07-07 12:51:24 +08:00
|
|
|
mhlo::BroadcastInDimOp, false> {
|
2020-07-07 04:57:00 +08:00
|
|
|
public:
|
|
|
|
using DataMovementOpConverter<HloBroadcastInDimConverter,
|
2020-07-07 12:51:24 +08:00
|
|
|
mhlo::BroadcastInDimOp,
|
2020-07-07 04:57:00 +08:00
|
|
|
false>::DataMovementOpConverter;
|
|
|
|
|
|
|
|
static SmallVector<AffineMap, 2> getIndexingMaps(
|
2020-10-24 03:22:21 +08:00
|
|
|
mhlo::BroadcastInDimOp broadcast_op, Builder* b) {
|
2020-12-15 02:46:04 +08:00
|
|
|
auto result_type = GetHloOpResultType<false>(broadcast_op);
|
2020-10-24 03:22:21 +08:00
|
|
|
auto operand_type =
|
|
|
|
broadcast_op.operand().getType().template cast<ShapedType>();
|
|
|
|
unsigned nloops = result_type.getRank();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// The input is a scalar, i.e. this is a scalar broadcast op.
|
2020-10-24 03:22:21 +08:00
|
|
|
if (operand_type.getRank() == 0) {
|
2020-07-07 04:57:00 +08:00
|
|
|
return {AffineMap::get(nloops, /*symbolCount=*/0, b->getContext()),
|
|
|
|
b->getMultiDimIdentityMap(nloops)};
|
|
|
|
}
|
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
auto operand_shape = operand_type.getShape();
|
|
|
|
SmallVector<AffineExpr, 4> dim_exprs;
|
|
|
|
dim_exprs.reserve(nloops);
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
if (broadcast_op.broadcast_dimensions()) {
|
2020-07-07 04:57:00 +08:00
|
|
|
for (const auto& broadcastDim :
|
2020-10-24 03:22:21 +08:00
|
|
|
enumerate(broadcast_op.broadcast_dimensions().getIntValues())) {
|
2020-07-07 04:57:00 +08:00
|
|
|
int size = broadcastDim.value().getSExtValue();
|
2020-10-24 03:22:21 +08:00
|
|
|
bool expansion_needed = operand_shape[broadcastDim.index()] == 1 &&
|
|
|
|
result_type.getShape()[size] != 1;
|
|
|
|
dim_exprs.push_back(expansion_needed ? b->getAffineConstantExpr(0)
|
|
|
|
: b->getAffineDimExpr(size));
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return {
|
2020-10-24 03:22:21 +08:00
|
|
|
AffineMap::get(nloops, /*symbolCount=*/0, dim_exprs, b->getContext()),
|
2020-07-07 04:57:00 +08:00
|
|
|
b->getMultiDimIdentityMap(nloops)};
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-01-12 02:11:39 +08:00
|
|
|
class HloDynamicBroadcastInDimConverter
|
|
|
|
: public OpConversionPattern<mhlo::DynamicBroadcastInDimOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<mhlo::DynamicBroadcastInDimOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::DynamicBroadcastInDimOp op, ArrayRef<Value> operands,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2021-03-19 18:51:14 +08:00
|
|
|
// If the input has a static shape we know exactly when the broadcast must
|
|
|
|
// expand (the dimension is 1, which also trivially expands to 1) or will
|
|
|
|
// never expand (the dimension is not 1). This means we can lower the
|
|
|
|
// broadcast just as we would lower a fully static broadcast and go directly
|
|
|
|
// to linalg.generic. This also covers the important case of broadcasting a
|
|
|
|
// scalar.
|
|
|
|
|
|
|
|
// Ideally the pattern (`mhlo.constant` -> `mhlo.dynamic_broadcast_in_dim`)
|
|
|
|
// should be converted to an Tensor-dialect op similar to TF ConstantLikeOp.
|
2021-01-12 02:11:39 +08:00
|
|
|
|
|
|
|
mhlo::DynamicBroadcastInDimOp::Adaptor adaptor(op);
|
|
|
|
Value operand = adaptor.operand();
|
|
|
|
auto operand_type = operand.getType().dyn_cast<RankedTensorType>();
|
2021-03-19 18:51:14 +08:00
|
|
|
if (!operand_type || !operand_type.hasStaticShape()) return failure();
|
2021-01-12 02:11:39 +08:00
|
|
|
|
|
|
|
Value shape = adaptor.output_dimensions();
|
|
|
|
auto shape_type = shape.getType().cast<RankedTensorType>();
|
|
|
|
int64_t result_rank = shape_type.getDimSize(0);
|
|
|
|
|
2021-03-01 19:03:03 +08:00
|
|
|
auto result_type = op.getType().dyn_cast<RankedTensorType>();
|
|
|
|
if (!result_type) return failure();
|
|
|
|
|
2021-01-12 02:11:39 +08:00
|
|
|
SmallVector<Value, 2> dyn_dims;
|
|
|
|
Location loc = op.getLoc();
|
|
|
|
for (int i = 0; i < result_rank; ++i) {
|
2021-03-01 19:03:03 +08:00
|
|
|
if (!result_type.isDynamicDim(i)) continue;
|
2021-01-12 02:11:39 +08:00
|
|
|
Value index = rewriter.create<ConstantIndexOp>(loc, i);
|
|
|
|
dyn_dims.push_back(rewriter.create<tensor::ExtractOp>(loc, shape, index));
|
|
|
|
}
|
|
|
|
|
|
|
|
int64_t nloops = result_type.getRank();
|
2021-03-19 18:51:14 +08:00
|
|
|
auto operand_shape = operand_type.getShape();
|
|
|
|
SmallVector<AffineExpr, 4> dim_exprs;
|
|
|
|
dim_exprs.reserve(nloops);
|
|
|
|
|
|
|
|
if (op.broadcast_dimensions()) {
|
|
|
|
for (const auto& broadcast_dim :
|
|
|
|
enumerate(op.broadcast_dimensions().getIntValues())) {
|
|
|
|
int64_t size = broadcast_dim.value().getSExtValue();
|
|
|
|
bool expansion_needed = operand_shape[broadcast_dim.index()] == 1;
|
|
|
|
dim_exprs.push_back(expansion_needed ? rewriter.getAffineConstantExpr(0)
|
|
|
|
: rewriter.getAffineDimExpr(size));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-12 02:11:39 +08:00
|
|
|
Value init = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, dyn_dims, result_type.getShape(), result_type.getElementType());
|
|
|
|
Operation* generic = rewriter.create<linalg::GenericOp>(
|
|
|
|
loc, TypeRange{init.getType()}, ValueRange{operand},
|
|
|
|
/*outputBuffers=*/ValueRange{init},
|
|
|
|
llvm::makeArrayRef(
|
2021-03-19 18:51:14 +08:00
|
|
|
{AffineMap::get(/*dimCount=*/nloops, /*symbolCount=*/0, dim_exprs,
|
2021-01-12 02:11:39 +08:00
|
|
|
rewriter.getContext()),
|
|
|
|
rewriter.getMultiDimIdentityMap(nloops)}),
|
|
|
|
GetNParallelLoopsAttrs(nloops),
|
|
|
|
[&](OpBuilder& nested_builder, Location nested_loc, ValueRange args) {
|
|
|
|
nested_builder.create<linalg::YieldOp>(loc, *args.begin());
|
|
|
|
});
|
|
|
|
rewriter.replaceOp(op, generic->getResults());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
class LhloBroadcastInDimConverter
|
2020-07-09 01:05:32 +08:00
|
|
|
: public OpConversionPattern<lmhlo::BroadcastInDimOp> {
|
2020-07-07 04:57:00 +08:00
|
|
|
public:
|
2020-07-09 01:05:32 +08:00
|
|
|
using OpConversionPattern<lmhlo::BroadcastInDimOp>::OpConversionPattern;
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
2020-07-09 01:05:32 +08:00
|
|
|
lmhlo::BroadcastInDimOp op, ArrayRef<Value> args,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2020-07-09 01:05:32 +08:00
|
|
|
lmhlo::BroadcastInDimOp::Adaptor operand_adaptor(args);
|
2020-07-07 04:57:00 +08:00
|
|
|
auto result_type = operand_adaptor.output().getType().cast<MemRefType>();
|
|
|
|
auto result_shape = result_type.getShape();
|
|
|
|
|
|
|
|
auto operand_and_dims = InsertReshapeIfNecessary(op, args, rewriter);
|
|
|
|
|
|
|
|
Value operand = std::get<0>(operand_and_dims);
|
|
|
|
auto broadcast_dims = std::get<1>(operand_and_dims);
|
|
|
|
|
|
|
|
auto loc = op.getLoc();
|
|
|
|
auto nloops = result_type.getRank();
|
|
|
|
auto operand_type = operand.getType().cast<MemRefType>();
|
|
|
|
|
|
|
|
// For a degenerate case, i.e. broadcasting with expansion of
|
|
|
|
// memref<1xELEMENT_TYPE>, the operand is not passed to `linalg.generic`.
|
|
|
|
// Instead the value is loaded and used directly in `linalg.yield`.
|
|
|
|
if (operand_type.getRank() == 1 &&
|
|
|
|
operand_type.getDimSize(0) <
|
|
|
|
result_type.getDimSize(broadcast_dims.front())) {
|
|
|
|
Value zero = rewriter.create<ConstantIndexOp>(loc, 0);
|
2021-03-17 04:31:59 +08:00
|
|
|
Value val = rewriter.create<memref::LoadOp>(loc, operand,
|
|
|
|
llvm::makeArrayRef({zero}));
|
2020-07-07 04:57:00 +08:00
|
|
|
rewriter.create<linalg::GenericOp>(
|
2020-09-23 00:06:55 +08:00
|
|
|
loc, /*inputs=*/ValueRange{},
|
|
|
|
/*outputBuffers=*/ValueRange{operand_adaptor.output()},
|
2020-07-07 04:57:00 +08:00
|
|
|
llvm::makeArrayRef(rewriter.getMultiDimIdentityMap(nloops)),
|
|
|
|
GetNParallelLoopsAttrs(nloops),
|
2020-10-24 03:22:21 +08:00
|
|
|
[&](OpBuilder& nested_builder, Location nested_loc, ValueRange args) {
|
|
|
|
nested_builder.create<linalg::YieldOp>(loc, val);
|
2020-07-07 04:57:00 +08:00
|
|
|
});
|
|
|
|
|
|
|
|
} else {
|
|
|
|
auto indexing_maps = getIndexingMaps(op, broadcast_dims, result_shape,
|
|
|
|
operand_type, &rewriter);
|
|
|
|
rewriter.create<linalg::GenericOp>(
|
2020-09-23 00:06:55 +08:00
|
|
|
loc, /*inputs=*/ValueRange{operand},
|
|
|
|
/*outputBuffers=*/ValueRange{operand_adaptor.output()}, indexing_maps,
|
2020-07-07 04:57:00 +08:00
|
|
|
GetNParallelLoopsAttrs(nloops),
|
2020-10-24 03:22:21 +08:00
|
|
|
[&](OpBuilder& nested_builder, Location nested_loc, ValueRange args) {
|
|
|
|
nested_builder.create<linalg::YieldOp>(loc, *args.begin());
|
2020-07-07 04:57:00 +08:00
|
|
|
});
|
|
|
|
}
|
|
|
|
rewriter.replaceOp(op, llvm::None);
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Inserts 'linalg.reshape' if there is a size-1 dim expansion.
|
|
|
|
std::pair<Value, SmallVector<int64_t, 2>> InsertReshapeIfNecessary(
|
2020-07-09 01:05:32 +08:00
|
|
|
lmhlo::BroadcastInDimOp op, ArrayRef<Value> args,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const {
|
2020-07-09 01:05:32 +08:00
|
|
|
lmhlo::BroadcastInDimOp::Adaptor operand_adaptor(args);
|
2020-07-07 04:57:00 +08:00
|
|
|
Value operand = operand_adaptor.operand();
|
|
|
|
auto operand_type = operand_adaptor.operand().getType().cast<MemRefType>();
|
|
|
|
auto operand_shape = operand_type.getShape();
|
|
|
|
|
|
|
|
Value result = operand_adaptor.output();
|
|
|
|
auto result_type = result.getType().cast<MemRefType>();
|
|
|
|
auto result_shape = result_type.getShape();
|
|
|
|
|
|
|
|
SmallVector<int64_t, 2> operand_strides;
|
|
|
|
int64_t operand_offset;
|
|
|
|
if (failed(getStridesAndOffset(operand_type, operand_strides,
|
|
|
|
operand_offset))) {
|
|
|
|
op.emitOpError() << "Failed to get offset and strides.";
|
|
|
|
}
|
|
|
|
|
|
|
|
SmallVector<int64_t, 2> new_shape, new_strides, broadcast_dims;
|
|
|
|
SmallVector<linalg::ReassociationIndices, 4> collapsed_dims_list;
|
|
|
|
linalg::ReassociationIndices collapsed_dims;
|
|
|
|
for (const auto& item :
|
|
|
|
enumerate(op.broadcast_dimensions().getIntValues())) {
|
|
|
|
size_t index = item.index();
|
|
|
|
int dim = item.value().getSExtValue();
|
|
|
|
|
|
|
|
collapsed_dims.push_back(index);
|
|
|
|
|
|
|
|
bool expansion_needed =
|
|
|
|
operand_shape[index] == 1 && result_shape[dim] != 1;
|
|
|
|
if (expansion_needed) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
new_shape.push_back(operand_shape[index]);
|
|
|
|
new_strides.push_back(operand_strides[index]);
|
|
|
|
broadcast_dims.push_back(dim);
|
|
|
|
|
|
|
|
collapsed_dims_list.push_back(collapsed_dims);
|
|
|
|
collapsed_dims.clear();
|
|
|
|
}
|
|
|
|
// If `collapsed_dims_list` is empty, then the memref has shape [1, ..., 1]
|
|
|
|
// and all dimensions need expansion. Such memref will be reshaped to a 1D
|
|
|
|
// memref with a single element. New shape and strides needs to be updated
|
|
|
|
// accordingly.
|
|
|
|
if (collapsed_dims_list.empty()) {
|
|
|
|
collapsed_dims_list.push_back({});
|
|
|
|
new_shape.push_back(1);
|
|
|
|
new_strides.push_back(1);
|
|
|
|
broadcast_dims.push_back(0);
|
|
|
|
}
|
|
|
|
for (const auto& dims : collapsed_dims) {
|
|
|
|
collapsed_dims_list.back().push_back(dims);
|
|
|
|
}
|
|
|
|
|
|
|
|
// `linalg.reshape` is inserted only if necessary, i.e. when the rank can be
|
|
|
|
// reduced.
|
|
|
|
if (new_shape.size() < operand_shape.size()) {
|
|
|
|
auto new_memref_type = MemRefType::get(
|
|
|
|
new_shape, operand_type.getElementType(),
|
|
|
|
makeStridedLinearLayoutMap(new_strides, operand_offset,
|
|
|
|
rewriter.getContext()));
|
|
|
|
operand = rewriter.create<linalg::ReshapeOp>(op.getLoc(), new_memref_type,
|
|
|
|
operand_adaptor.operand(),
|
|
|
|
collapsed_dims_list);
|
|
|
|
}
|
|
|
|
return std::make_pair(operand, broadcast_dims);
|
|
|
|
}
|
|
|
|
|
2020-07-09 01:05:32 +08:00
|
|
|
SmallVector<AffineMap, 2> getIndexingMaps(lmhlo::BroadcastInDimOp op,
|
2020-10-24 03:22:21 +08:00
|
|
|
ArrayRef<int64_t> broadcast_dims,
|
|
|
|
ArrayRef<int64_t> result_shape,
|
|
|
|
MemRefType operand_type,
|
2020-07-07 04:57:00 +08:00
|
|
|
Builder* b) const {
|
2020-10-24 03:22:21 +08:00
|
|
|
unsigned nloops = result_shape.size();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// The input is a scalar, i.e. this is a scalar broadcast op.
|
2020-10-24 03:22:21 +08:00
|
|
|
if (operand_type.getRank() == 0) {
|
2020-07-07 04:57:00 +08:00
|
|
|
return {AffineMap::get(nloops, /*symbolCount=*/0, b->getContext()),
|
|
|
|
b->getMultiDimIdentityMap(nloops)};
|
|
|
|
}
|
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
auto operand_shape = operand_type.getShape();
|
|
|
|
SmallVector<AffineExpr, 4> dim_exprs;
|
|
|
|
dim_exprs.reserve(nloops);
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
for (const auto& broadcast_dim : llvm::enumerate(broadcast_dims)) {
|
|
|
|
int size = broadcast_dim.value();
|
2020-07-07 04:57:00 +08:00
|
|
|
bool expansion_needed =
|
2020-10-24 03:22:21 +08:00
|
|
|
operand_shape[broadcast_dim.index()] == 1 && result_shape[size] != 1;
|
2020-07-07 04:57:00 +08:00
|
|
|
if (expansion_needed) {
|
|
|
|
op.emitOpError(
|
|
|
|
"BroadcastInDimOp lowering to Linalg does not support size-1 "
|
|
|
|
"dimensions expansion.");
|
|
|
|
}
|
2020-10-24 03:22:21 +08:00
|
|
|
dim_exprs.push_back(b->getAffineDimExpr(size));
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
return {
|
2020-10-24 03:22:21 +08:00
|
|
|
AffineMap::get(nloops, /*symbolCount=*/0, dim_exprs, b->getContext()),
|
2020-07-07 04:57:00 +08:00
|
|
|
b->getMultiDimIdentityMap(nloops)};
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class TransposeConverter
|
|
|
|
: public DataMovementOpConverter<TransposeConverter<OpTy, isLHLO>, OpTy,
|
|
|
|
isLHLO> {
|
|
|
|
public:
|
|
|
|
using DataMovementOpConverter<TransposeConverter<OpTy, isLHLO>, OpTy,
|
|
|
|
isLHLO>::DataMovementOpConverter;
|
|
|
|
static SmallVector<AffineMap, 2> getIndexingMaps(OpTy op, Builder* b) {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto result_type =
|
2020-12-15 02:46:04 +08:00
|
|
|
GetHloOpResultType<isLHLO>(op).template cast<ShapedType>();
|
2020-10-24 03:22:21 +08:00
|
|
|
auto nloops = result_type.getRank();
|
|
|
|
SmallVector<AffineExpr, 2> input_exprs;
|
|
|
|
input_exprs.resize(result_type.getRank());
|
2020-07-07 04:57:00 +08:00
|
|
|
for (auto permutation : llvm::enumerate(op.permutation())) {
|
2020-10-24 03:22:21 +08:00
|
|
|
input_exprs[permutation.value().getZExtValue()] =
|
2020-07-07 04:57:00 +08:00
|
|
|
b->getAffineDimExpr(permutation.index());
|
|
|
|
}
|
|
|
|
return {
|
2020-10-24 03:22:21 +08:00
|
|
|
AffineMap::get(nloops, /*symbolCount=*/0, input_exprs, b->getContext()),
|
2020-07-07 04:57:00 +08:00
|
|
|
b->getMultiDimIdentityMap(nloops)};
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
// Converts reshape ops that can be proven to be either a collapse of dimensions
|
|
|
|
// or expansion of dimensions of the operand.
|
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class ReshapeOpConverter : public OpConversionPattern<OpTy> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
2020-10-24 03:22:21 +08:00
|
|
|
OpTy reshape_op, ArrayRef<Value> args,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2020-12-15 02:46:04 +08:00
|
|
|
if (!VerifyHloOpBufferOrTensorSemantics<isLHLO>(reshape_op))
|
2020-07-07 04:57:00 +08:00
|
|
|
return failure();
|
2021-01-28 21:36:37 +08:00
|
|
|
typename OpTy::Adaptor operands(args);
|
2020-10-24 03:22:21 +08:00
|
|
|
ShapedType operand_type =
|
2021-01-28 21:36:37 +08:00
|
|
|
operands.operand().getType().template cast<ShapedType>();
|
2020-12-15 02:46:04 +08:00
|
|
|
ShapedType result_type = GetHloOpResultType<isLHLO>(reshape_op);
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
if (!operand_type.hasStaticShape() || !result_type.hasStaticShape())
|
2020-07-07 04:57:00 +08:00
|
|
|
return failure();
|
|
|
|
|
|
|
|
// Compute the reassociation maps for the linalg operation.
|
2020-10-24 03:22:21 +08:00
|
|
|
ArrayRef<int64_t> src_shape =
|
|
|
|
(operand_type.getRank() > result_type.getRank()
|
|
|
|
? operand_type.getShape()
|
|
|
|
: result_type.getShape());
|
|
|
|
ArrayRef<int64_t> dst_shape =
|
|
|
|
(operand_type.getRank() > result_type.getRank()
|
|
|
|
? result_type.getShape()
|
|
|
|
: operand_type.getShape());
|
|
|
|
unsigned curr_src_dim = 0, curr_dst_dim = 0;
|
|
|
|
SmallVector<linalg::ReassociationExprs, 4> reassociation_map(
|
|
|
|
dst_shape.size());
|
2021-01-28 21:36:37 +08:00
|
|
|
|
|
|
|
// First scan all dimensions in the source shapes to see whether we have a
|
|
|
|
// perfect case where consecutive dimensions in source are collapsed. For
|
|
|
|
// such case we can just generate one single linalg.reshape.
|
|
|
|
bool is_collapsing_source = true;
|
2020-10-24 03:22:21 +08:00
|
|
|
while (curr_src_dim < src_shape.size() && curr_dst_dim < dst_shape.size()) {
|
|
|
|
int64_t dst_size = dst_shape[curr_dst_dim];
|
|
|
|
int64_t src_size = src_shape[curr_src_dim];
|
|
|
|
while (src_size < dst_size && curr_src_dim < src_shape.size()) {
|
|
|
|
reassociation_map[curr_dst_dim].push_back(
|
|
|
|
rewriter.getAffineDimExpr(curr_src_dim++));
|
|
|
|
src_size *= src_shape[curr_src_dim];
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2020-10-24 03:22:21 +08:00
|
|
|
if (src_size == dst_size) {
|
|
|
|
reassociation_map[curr_dst_dim].push_back(
|
|
|
|
rewriter.getAffineDimExpr(curr_src_dim++));
|
|
|
|
// If the next dim in dst_shape is not 1, treat subsequent dims in
|
|
|
|
// src_shape which are 1 to be collapsed.
|
|
|
|
if (curr_dst_dim == dst_shape.size() - 1 ||
|
|
|
|
dst_shape[curr_dst_dim + 1] != 1) {
|
|
|
|
while (curr_src_dim < src_shape.size() &&
|
|
|
|
src_shape[curr_src_dim] == 1) {
|
|
|
|
reassociation_map[curr_dst_dim].push_back(
|
|
|
|
rewriter.getAffineDimExpr(curr_src_dim++));
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
} else {
|
2021-01-28 21:36:37 +08:00
|
|
|
is_collapsing_source = false;
|
2020-08-22 14:26:35 +08:00
|
|
|
break;
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2020-10-24 03:22:21 +08:00
|
|
|
curr_dst_dim++;
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2021-03-11 21:28:51 +08:00
|
|
|
// Rank 0 can always use the direct lowering.
|
|
|
|
if (!src_shape.empty() && !dst_shape.empty() &&
|
|
|
|
(curr_src_dim != src_shape.size() || curr_dst_dim != dst_shape.size()))
|
2021-01-28 21:36:37 +08:00
|
|
|
is_collapsing_source = false;
|
2020-08-22 14:26:35 +08:00
|
|
|
|
2021-01-28 21:36:37 +08:00
|
|
|
// Otherwise, we need to first reduce all source dimensions into one and
|
|
|
|
// then expand to the destination dimensions.
|
|
|
|
if (!is_collapsing_source) {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto get_identity_exprs = [&rewriter](int n) {
|
2020-08-22 14:26:35 +08:00
|
|
|
SmallVector<AffineExpr, 4> exprs;
|
|
|
|
for (int i = 0; i < n; ++i)
|
|
|
|
exprs.push_back(rewriter.getAffineDimExpr(i));
|
|
|
|
return exprs;
|
|
|
|
};
|
2020-10-24 03:22:21 +08:00
|
|
|
Location loc = reshape_op.getLoc();
|
|
|
|
int64_t total_elems = std::accumulate(src_shape.begin(), src_shape.end(),
|
|
|
|
1, std::multiplies<int64_t>());
|
|
|
|
auto elem_type = operand_type.getElementType();
|
|
|
|
SmallVector<linalg::ReassociationExprs, 4> collapsing_map = {
|
2021-01-28 21:36:37 +08:00
|
|
|
// Use operand_type here because we need to collapse all operands
|
|
|
|
// dimensions.
|
|
|
|
get_identity_exprs(operand_type.getShape().size())};
|
2020-10-24 03:22:21 +08:00
|
|
|
SmallVector<linalg::ReassociationExprs, 4> expanding_map = {
|
2021-01-28 21:36:37 +08:00
|
|
|
// Use result_type here because we need to expand to all result
|
|
|
|
// dimensions.
|
|
|
|
get_identity_exprs(result_type.getShape().size())};
|
2020-08-22 14:26:35 +08:00
|
|
|
|
|
|
|
if (isLHLO) {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto collapsed_type = MemRefType::get({total_elems}, elem_type);
|
|
|
|
Value collapsed_op = rewriter.create<linalg::ReshapeOp>(
|
|
|
|
loc, collapsed_type, args[0], collapsing_map);
|
|
|
|
Value reshape_buffer = rewriter.create<linalg::ReshapeOp>(
|
|
|
|
loc, result_type, collapsed_op, expanding_map);
|
2021-02-15 20:17:20 +08:00
|
|
|
rewriter.replaceOpWithNewOp<linalg::CopyOp>(reshape_op, reshape_buffer,
|
|
|
|
args[1]);
|
2020-08-22 14:26:35 +08:00
|
|
|
} else {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto collapsed_type = RankedTensorType::get({total_elems}, elem_type);
|
|
|
|
Value collapsed_op = rewriter.create<linalg::TensorReshapeOp>(
|
|
|
|
loc, collapsed_type, args[0], collapsing_map);
|
2020-08-22 14:26:35 +08:00
|
|
|
rewriter.replaceOpWithNewOp<linalg::TensorReshapeOp>(
|
2020-10-24 03:22:21 +08:00
|
|
|
reshape_op, result_type, collapsed_op, expanding_map);
|
2020-08-22 14:26:35 +08:00
|
|
|
}
|
|
|
|
return success();
|
|
|
|
}
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
if (isLHLO) {
|
2020-10-24 03:22:21 +08:00
|
|
|
Value reshape_buffer = rewriter.create<linalg::ReshapeOp>(
|
|
|
|
reshape_op.getLoc(), result_type, args[0], reassociation_map);
|
2021-02-15 20:17:20 +08:00
|
|
|
rewriter.replaceOpWithNewOp<linalg::CopyOp>(reshape_op, reshape_buffer,
|
|
|
|
args[1]);
|
2020-07-07 04:57:00 +08:00
|
|
|
} else {
|
|
|
|
rewriter.replaceOpWithNewOp<linalg::TensorReshapeOp>(
|
2020-10-24 03:22:21 +08:00
|
|
|
reshape_op, result_type, args[0], reassociation_map);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-24 00:18:01 +08:00
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class IotaConverter : public OpConversionPattern<OpTy> {
|
2020-07-07 04:57:00 +08:00
|
|
|
public:
|
2020-07-24 00:18:01 +08:00
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
2020-10-24 03:22:21 +08:00
|
|
|
OpTy iota_op, ArrayRef<Value> args,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2020-12-15 02:46:04 +08:00
|
|
|
ShapedType result_shaped_type = GetHloOpResultType<isLHLO>(iota_op);
|
2020-10-24 03:22:21 +08:00
|
|
|
if (!result_shaped_type) return failure();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-10-24 03:22:21 +08:00
|
|
|
auto result_element_type = result_shaped_type.getElementType();
|
|
|
|
if (!result_element_type.isSignlessIntOrFloat()) return failure();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
// Construct the indexing maps needed for linalg.generic ops.
|
2020-10-24 03:22:21 +08:00
|
|
|
unsigned nloops = result_shaped_type.getRank();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-12-24 15:53:08 +08:00
|
|
|
Location loc = iota_op.getLoc();
|
2021-03-12 00:30:08 +08:00
|
|
|
// If this is a dynamic iota, the first argument will be a shape tensor.
|
|
|
|
Value shape_tensor = args.size() > (isLHLO ? 1 : 0) ? args[0] : nullptr;
|
|
|
|
auto dyn_sizes =
|
|
|
|
isLHLO
|
|
|
|
? SmallVector<Value, 2>()
|
|
|
|
: ExtractDynamicSizes(
|
|
|
|
rewriter, loc, GetResultValue<isLHLO>(iota_op), shape_tensor);
|
2020-10-24 03:22:21 +08:00
|
|
|
auto linalg_op = rewriter.create<linalg::IndexedGenericOp>(
|
2020-12-24 15:53:08 +08:00
|
|
|
loc,
|
2020-09-23 00:06:55 +08:00
|
|
|
/*resultTensorTypes=*/
|
2020-10-24 03:22:21 +08:00
|
|
|
isLHLO ? ArrayRef<Type>{} : ArrayRef<Type>{result_shaped_type},
|
2020-09-23 00:06:55 +08:00
|
|
|
/*inputs=*/ValueRange{},
|
2020-12-24 15:53:08 +08:00
|
|
|
/*outputBuffers=*/
|
2021-03-12 00:30:08 +08:00
|
|
|
isLHLO ? ValueRange{args.back()}
|
2021-02-04 07:02:21 +08:00
|
|
|
: ValueRange{GetInitTensor(rewriter, loc, result_shaped_type,
|
|
|
|
dyn_sizes)},
|
2020-07-07 04:57:00 +08:00
|
|
|
llvm::makeArrayRef(rewriter.getMultiDimIdentityMap(nloops)),
|
|
|
|
GetNParallelLoopsAttrs(nloops),
|
2020-10-24 03:22:21 +08:00
|
|
|
[&](OpBuilder& nested_builder, Location nested_loc, ValueRange ivs,
|
2020-07-07 04:57:00 +08:00
|
|
|
ValueRange args) {
|
2020-10-24 03:22:21 +08:00
|
|
|
Value cast_op = nested_builder.create<IndexCastOp>(
|
|
|
|
nested_loc, ivs[iota_op.iota_dimension()],
|
|
|
|
nested_builder.getIntegerType(
|
|
|
|
result_element_type.getIntOrFloatBitWidth()));
|
|
|
|
if (result_element_type.template isa<FloatType>()) {
|
|
|
|
cast_op = nested_builder.create<SIToFPOp>(nested_loc, cast_op,
|
|
|
|
result_element_type);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2020-10-24 03:22:21 +08:00
|
|
|
nested_builder.create<linalg::YieldOp>(nested_loc, cast_op);
|
2020-07-07 04:57:00 +08:00
|
|
|
});
|
2020-07-24 00:18:01 +08:00
|
|
|
if (isLHLO)
|
2020-10-24 03:22:21 +08:00
|
|
|
rewriter.replaceOp(iota_op, llvm::None);
|
2020-07-24 00:18:01 +08:00
|
|
|
else
|
2020-10-24 03:22:21 +08:00
|
|
|
rewriter.replaceOp(iota_op, linalg_op.result_tensors());
|
2020-07-07 04:57:00 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-04-22 01:00:12 +08:00
|
|
|
/// Converts mhlo.concatenate operation to a linalg.generic op.
|
|
|
|
struct ConcatenateConverter : public OpConversionPattern<mhlo::ConcatenateOp> {
|
|
|
|
using OpConversionPattern<mhlo::ConcatenateOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::ConcatenateOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const override {
|
|
|
|
// Shortcut the one-operand case, simplifies code below.
|
|
|
|
if (args.size() == 1) {
|
|
|
|
rewriter.replaceOp(op, args[0]);
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
|
|
|
|
auto result_type = op.getResult().getType().dyn_cast<RankedTensorType>();
|
|
|
|
if (!result_type) return failure();
|
|
|
|
|
|
|
|
ImplicitLocOpBuilder b(op.getLoc(), rewriter);
|
|
|
|
uint64_t dim = op.dimension();
|
|
|
|
int64_t rank = result_type.getRank();
|
|
|
|
Value zero = b.create<ConstantIndexOp>(0);
|
|
|
|
SmallVector<Value, 3> sizes;
|
|
|
|
for (int64_t i = 0; i < rank; ++i) {
|
|
|
|
sizes.push_back(i == dim ? Value() : b.create<memref::DimOp>(args[0], i));
|
|
|
|
}
|
|
|
|
|
|
|
|
// Calculate the size of the concatenated dimension.
|
|
|
|
Value result_dim_size;
|
|
|
|
for (auto arg : args) {
|
|
|
|
Value size = b.create<memref::DimOp>(arg, dim);
|
|
|
|
result_dim_size =
|
|
|
|
result_dim_size ? b.create<AddIOp>(result_dim_size, size) : size;
|
|
|
|
}
|
|
|
|
sizes[dim] = result_dim_size;
|
|
|
|
|
|
|
|
// Allocate the output tensor with init_tensor.
|
|
|
|
SmallVector<Value, 3> dyn_sizes;
|
|
|
|
for (int64_t i = 0; i < rank; ++i) {
|
|
|
|
if (result_type.isDynamicDim(i)) dyn_sizes.push_back(sizes[i]);
|
|
|
|
}
|
|
|
|
Value result = b.create<linalg::InitTensorOp>(
|
|
|
|
dyn_sizes, result_type.getShape(), result_type.getElementType());
|
|
|
|
|
|
|
|
// Generate a generic op to gather the elements of the concatenate. This is
|
|
|
|
// awkward standalone but allows fusion with other generic ops.
|
|
|
|
unsigned nloops = result_type.getRank();
|
|
|
|
auto linalg_op = b.create<linalg::IndexedGenericOp>(
|
|
|
|
/*resultTensorTypes=*/result_type,
|
|
|
|
/*inputs=*/ValueRange{}, /*outputBuffers=*/result,
|
|
|
|
llvm::makeArrayRef(rewriter.getMultiDimIdentityMap(nloops)),
|
|
|
|
GetNParallelLoopsAttrs(nloops),
|
|
|
|
[&](OpBuilder& nested_builder, Location loc, ValueRange ivs,
|
|
|
|
ValueRange) {
|
|
|
|
OpBuilder b = nested_builder;
|
|
|
|
Value concat_dim_size = zero;
|
|
|
|
Value result;
|
|
|
|
auto extract_indices = llvm::to_vector<4>(ivs);
|
|
|
|
for (const Value& arg : args) {
|
|
|
|
Value new_concat_dim_size;
|
|
|
|
scf::IfOp if_op;
|
|
|
|
if (&arg != &args.back()) {
|
|
|
|
// Calculate how far along we have iterated along the concatenate
|
|
|
|
// dimension. That way we can tell which input to select.
|
|
|
|
new_concat_dim_size = b.create<AddIOp>(
|
|
|
|
loc, concat_dim_size, b.create<memref::DimOp>(loc, arg, dim));
|
|
|
|
Value cmp = b.create<CmpIOp>(loc, rewriter.getI1Type(),
|
|
|
|
CmpIPredicate::ult, ivs[dim],
|
|
|
|
new_concat_dim_size);
|
|
|
|
if_op = b.create<scf::IfOp>(loc, result_type.getElementType(),
|
|
|
|
cmp, true);
|
|
|
|
if (result) {
|
|
|
|
b.create<scf::YieldOp>(loc, if_op->getResults()[0]);
|
|
|
|
} else {
|
|
|
|
result = if_op->getResults()[0];
|
|
|
|
}
|
|
|
|
|
|
|
|
b = if_op.getThenBodyBuilder(b.getListener());
|
|
|
|
}
|
|
|
|
|
|
|
|
// Now adjust the index for the concatenated dimension to fit into
|
|
|
|
// the selected tensor and do an extract at that position.
|
|
|
|
extract_indices[dim] =
|
|
|
|
b.create<SubIOp>(loc, ivs[dim], concat_dim_size);
|
|
|
|
Value extract =
|
|
|
|
b.create<tensor::ExtractOp>(loc, arg, extract_indices);
|
|
|
|
b.create<scf::YieldOp>(loc, extract);
|
|
|
|
|
|
|
|
if (if_op) {
|
|
|
|
b = if_op.getElseBodyBuilder(b.getListener());
|
|
|
|
concat_dim_size = new_concat_dim_size;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
nested_builder.create<linalg::YieldOp>(loc, result);
|
|
|
|
});
|
|
|
|
rewriter.replaceOp(op, linalg_op.result_tensors());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-12-10 15:24:23 +08:00
|
|
|
template <typename OpTy>
|
|
|
|
class ConstConverter : public OpConversionPattern<OpTy> {
|
2020-07-07 04:57:00 +08:00
|
|
|
public:
|
2020-12-10 15:24:23 +08:00
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
2020-12-10 15:24:23 +08:00
|
|
|
OpTy const_op, ArrayRef<Value> /*args*/,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2020-12-10 15:24:23 +08:00
|
|
|
Location loc = const_op.getLoc();
|
|
|
|
auto value_attr = const_op.value().template cast<DenseElementsAttr>();
|
2020-10-24 03:22:21 +08:00
|
|
|
if (value_attr.getType().getRank() != 0) return failure();
|
2020-12-10 15:24:23 +08:00
|
|
|
ReplaceConstOp(loc, const_op, value_attr, rewriter);
|
2020-07-07 04:57:00 +08:00
|
|
|
return success();
|
|
|
|
}
|
2020-12-10 15:24:23 +08:00
|
|
|
|
|
|
|
private:
|
|
|
|
void ReplaceConstOp(Location loc, mhlo::ConstOp op,
|
|
|
|
DenseElementsAttr value_attr,
|
|
|
|
ConversionPatternRewriter& rewriter) const {
|
|
|
|
Value std_tensor_const = rewriter.create<mlir::ConstantOp>(loc, value_attr);
|
|
|
|
rewriter.replaceOp(op, {std_tensor_const});
|
|
|
|
}
|
|
|
|
void ReplaceConstOp(Location loc, lmhlo::ConstOp op,
|
|
|
|
DenseElementsAttr value_attr,
|
|
|
|
ConversionPatternRewriter& rewriter) const {
|
|
|
|
Value std_scalar_const =
|
|
|
|
rewriter.create<mlir::ConstantOp>(loc, value_attr.getValue({}));
|
|
|
|
rewriter.create<mlir::AffineStoreOp>(loc, std_scalar_const, op.getOperand(),
|
|
|
|
llvm::None);
|
|
|
|
rewriter.eraseOp(op);
|
|
|
|
}
|
2020-07-07 04:57:00 +08:00
|
|
|
};
|
|
|
|
|
2020-10-29 07:37:38 +08:00
|
|
|
class ReduceConverter : public OpConversionPattern<lmhlo::ReduceOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<lmhlo::ReduceOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
lmhlo::ReduceOp reduce_op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
auto loc = reduce_op.getLoc();
|
|
|
|
lmhlo::ReduceOp::Adaptor adaptor(args);
|
|
|
|
auto operand_shape =
|
2021-04-15 03:05:33 +08:00
|
|
|
adaptor.inputs()[0].getType().template dyn_cast<ShapedType>();
|
2020-10-29 07:37:38 +08:00
|
|
|
if (!operand_shape || !operand_shape.hasRank()) {
|
|
|
|
emitError(loc, "lhlo to linalg conversion expects known-rank args");
|
|
|
|
return failure();
|
|
|
|
}
|
|
|
|
|
|
|
|
// First fill the output buffer with the init value.
|
2021-03-17 04:31:59 +08:00
|
|
|
Value init_value =
|
|
|
|
rewriter.create<memref::LoadOp>(loc, adaptor.init_values()[0]);
|
2020-10-29 07:37:38 +08:00
|
|
|
rewriter.create<linalg::FillOp>(loc, adaptor.out()[0], init_value);
|
|
|
|
|
|
|
|
DenseIntElementsAttr dimensions_attr = reduce_op.dimensions();
|
|
|
|
SmallVector<int, 4> reduction_dims;
|
|
|
|
for (const auto& dim : dimensions_attr.getIntValues()) {
|
|
|
|
reduction_dims.push_back(dim.getSExtValue());
|
|
|
|
}
|
|
|
|
|
|
|
|
SmallVector<AffineExpr, 2> src_exprs;
|
|
|
|
SmallVector<AffineExpr, 2> dst_exprs;
|
|
|
|
SmallVector<StringRef, 4> types;
|
|
|
|
for (int i = 0, rank = operand_shape.getRank(); i != rank; ++i) {
|
|
|
|
bool is_reduced = llvm::is_contained(reduction_dims, i);
|
|
|
|
types.push_back(is_reduced ? getReductionIteratorTypeName()
|
|
|
|
: getParallelIteratorTypeName());
|
|
|
|
|
|
|
|
src_exprs.push_back(mlir::getAffineDimExpr(i, rewriter.getContext()));
|
|
|
|
if (!is_reduced) {
|
|
|
|
dst_exprs.push_back(mlir::getAffineDimExpr(i, rewriter.getContext()));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
auto maps = AffineMap::inferFromExprList({src_exprs, dst_exprs});
|
|
|
|
|
|
|
|
auto linalg_op = rewriter.create<linalg::GenericOp>(
|
|
|
|
loc, /*resultTensorTypes=*/ArrayRef<Type>{},
|
2021-04-15 03:05:33 +08:00
|
|
|
/*inputs=*/adaptor.inputs(), /*outputBuffers=*/adaptor.out(), maps,
|
2020-12-24 15:53:08 +08:00
|
|
|
types);
|
2020-12-16 00:56:01 +08:00
|
|
|
rewriter.inlineRegionBefore(reduce_op.body(), linalg_op.region(),
|
|
|
|
linalg_op.region().end());
|
2020-10-29 07:37:38 +08:00
|
|
|
{
|
|
|
|
OpBuilder::InsertionGuard region_guard(rewriter);
|
|
|
|
Block* block = linalg_op.getBody();
|
|
|
|
rewriter.setInsertionPoint(&block->front());
|
|
|
|
|
|
|
|
// The incoming region is operating on buffers, while linalg.generic
|
|
|
|
// expects scalar SSA values. Add some allocs around the original op to
|
|
|
|
// make it compatible.
|
|
|
|
auto arg_type = block->getArgument(0).getType().cast<MemRefType>();
|
2021-03-17 04:31:59 +08:00
|
|
|
Value alloc_a = rewriter.create<memref::AllocaOp>(loc, arg_type);
|
|
|
|
Value alloc_b = rewriter.create<memref::AllocaOp>(loc, arg_type);
|
|
|
|
Value alloc_res = rewriter.create<memref::AllocaOp>(loc, arg_type);
|
2020-10-29 07:37:38 +08:00
|
|
|
|
|
|
|
// Now turn the existing signature
|
|
|
|
// (memref<X>, memref<X>, memref<X>) -> ()
|
|
|
|
// into
|
|
|
|
// (X, X) -> X
|
|
|
|
TypeConverter::SignatureConversion signature_converter(3);
|
|
|
|
signature_converter.remapInput(0, alloc_a);
|
|
|
|
signature_converter.remapInput(1, alloc_b);
|
|
|
|
signature_converter.remapInput(2, alloc_res);
|
|
|
|
signature_converter.addInputs(
|
|
|
|
{arg_type.getElementType(), arg_type.getElementType()});
|
|
|
|
Block* entry_block = rewriter.applySignatureConversion(
|
|
|
|
&linalg_op.region(), signature_converter);
|
|
|
|
|
|
|
|
// Store the arguments into the newly allocated buffers.
|
|
|
|
rewriter.setInsertionPointAfter(alloc_res.getDefiningOp());
|
2021-03-17 04:31:59 +08:00
|
|
|
rewriter.create<memref::StoreOp>(loc, entry_block->getArgument(0),
|
|
|
|
alloc_a);
|
|
|
|
rewriter.create<memref::StoreOp>(loc, entry_block->getArgument(1),
|
|
|
|
alloc_b);
|
2020-10-29 07:37:38 +08:00
|
|
|
rewriter.replaceOp(entry_block->getTerminator(), {});
|
|
|
|
|
|
|
|
// Load & yield the result.
|
|
|
|
rewriter.setInsertionPointToEnd(entry_block);
|
2021-03-17 04:31:59 +08:00
|
|
|
auto load_res = rewriter.create<memref::LoadOp>(loc, alloc_res);
|
2020-10-29 07:37:38 +08:00
|
|
|
rewriter.create<linalg::YieldOp>(loc, ValueRange{load_res});
|
|
|
|
}
|
|
|
|
|
|
|
|
rewriter.replaceOp(reduce_op, linalg_op.getOperation()->getResults());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
// TODO(b/156787842): Support the lowering for dynamic shapes.
|
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class ReverseConverter
|
|
|
|
: public DataMovementOpConverter<ReverseConverter<OpTy, isLHLO>, OpTy,
|
|
|
|
isLHLO> {
|
|
|
|
public:
|
|
|
|
using DataMovementOpConverter<ReverseConverter<OpTy, isLHLO>, OpTy,
|
|
|
|
isLHLO>::DataMovementOpConverter;
|
|
|
|
static SmallVector<AffineMap, 2> getIndexingMaps(OpTy op, Builder* b) {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto result_type =
|
2020-12-15 02:46:04 +08:00
|
|
|
GetHloOpResultType<isLHLO>(op).template cast<ShapedType>();
|
2020-10-24 03:22:21 +08:00
|
|
|
auto nloops = result_type.getRank();
|
|
|
|
SmallVector<AffineExpr, 2> input_exprs;
|
|
|
|
input_exprs.reserve(nloops);
|
2020-07-07 04:57:00 +08:00
|
|
|
for (int i = 0; i < nloops; ++i)
|
2020-10-24 03:22:21 +08:00
|
|
|
input_exprs.push_back(b->getAffineDimExpr(i));
|
2020-07-07 04:57:00 +08:00
|
|
|
for (auto dim : op.dimensions()) {
|
|
|
|
int i = dim.getZExtValue();
|
2020-10-24 03:22:21 +08:00
|
|
|
if (result_type.isDynamicDim(i)) return {};
|
|
|
|
int n = result_type.getShape()[i];
|
|
|
|
input_exprs[i] = b->getAffineConstantExpr(n - 1) - input_exprs[i];
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
return {
|
2020-10-24 03:22:21 +08:00
|
|
|
AffineMap::get(nloops, /*symbolCount=*/0, input_exprs, b->getContext()),
|
2020-07-07 04:57:00 +08:00
|
|
|
b->getMultiDimIdentityMap(nloops)};
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-02-25 01:04:32 +08:00
|
|
|
template <typename OpTy, bool isLHLO = true>
|
|
|
|
class SliceConverter : public OpConversionPattern<OpTy> {
|
2020-07-07 04:57:00 +08:00
|
|
|
public:
|
2021-02-25 01:04:32 +08:00
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
2021-02-25 01:04:32 +08:00
|
|
|
OpTy slice_op, ArrayRef<Value> args,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
2020-10-24 03:22:21 +08:00
|
|
|
auto loc = slice_op.getLoc();
|
2021-02-25 01:04:32 +08:00
|
|
|
auto arg_type = args[0].getType().template dyn_cast<ShapedType>();
|
2020-10-24 03:22:21 +08:00
|
|
|
if (!arg_type || !arg_type.hasRank()) {
|
2020-07-07 04:57:00 +08:00
|
|
|
emitError(loc, "lhlo to linalg conversion expects known-rank args");
|
|
|
|
return failure();
|
|
|
|
}
|
|
|
|
|
2021-02-05 05:41:18 +08:00
|
|
|
SmallVector<OpFoldResult, 3> offsets, sizes, strides;
|
2020-10-24 03:22:21 +08:00
|
|
|
for (int i = 0, e = arg_type.getRank(); i < e; ++i) {
|
2021-02-05 05:41:18 +08:00
|
|
|
offsets.push_back(rewriter.getI64IntegerAttr(
|
2021-02-25 01:04:32 +08:00
|
|
|
slice_op.start_indices().template getValue<int64_t>(i)));
|
2021-02-05 05:41:18 +08:00
|
|
|
sizes.push_back(rewriter.getI64IntegerAttr(
|
2021-02-25 01:04:32 +08:00
|
|
|
slice_op.limit_indices().template getValue<int64_t>(i) -
|
|
|
|
slice_op.start_indices().template getValue<int64_t>(i)));
|
|
|
|
strides.push_back(rewriter.getI64IntegerAttr(
|
|
|
|
slice_op.strides().template getValue<int64_t>(i)));
|
|
|
|
}
|
|
|
|
if (isLHLO) {
|
2021-03-17 04:31:59 +08:00
|
|
|
auto linalg_op = rewriter.create<memref::SubViewOp>(loc, args[0], offsets,
|
|
|
|
sizes, strides);
|
2021-02-25 01:04:32 +08:00
|
|
|
rewriter.create<linalg::CopyOp>(loc, linalg_op, args[1]);
|
|
|
|
rewriter.eraseOp(slice_op);
|
|
|
|
} else {
|
|
|
|
rewriter.replaceOpWithNewOp<SubTensorOp>(slice_op, args[0], offsets,
|
|
|
|
sizes, strides);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-04-13 01:34:01 +08:00
|
|
|
class DynamicSliceConverter : public OpConversionPattern<mhlo::DynamicSliceOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<mhlo::DynamicSliceOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::DynamicSliceOp dynamic_slice_op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
auto loc = dynamic_slice_op.getLoc();
|
|
|
|
mhlo::DynamicSliceOp::Adaptor adaptor(args);
|
|
|
|
auto arg_type = adaptor.operand().getType().dyn_cast<ShapedType>();
|
|
|
|
if (!arg_type || !arg_type.hasRank()) {
|
|
|
|
return rewriter.notifyMatchFailure(dynamic_slice_op,
|
|
|
|
"require known-rank args");
|
|
|
|
}
|
|
|
|
|
|
|
|
auto index_type = rewriter.getIndexType();
|
|
|
|
SmallVector<OpFoldResult, 3> start_indices, sizes;
|
|
|
|
Value zero = rewriter.create<ConstantOp>(
|
|
|
|
loc, rewriter.getZeroAttr(adaptor.start_indices()[0]
|
|
|
|
.getType()
|
|
|
|
.cast<RankedTensorType>()
|
|
|
|
.getElementType()));
|
|
|
|
for (auto en : llvm::enumerate(
|
|
|
|
llvm::zip(adaptor.start_indices(),
|
|
|
|
dynamic_slice_op.slice_sizes().getValues<int64_t>()))) {
|
|
|
|
int64_t size = std::get<1>(en.value());
|
|
|
|
sizes.push_back(rewriter.getI64IntegerAttr(size));
|
|
|
|
|
|
|
|
// By mhlo.DynamicSlice definition:
|
|
|
|
// `start_indices[i] = clamp(start_indices[i],
|
|
|
|
// 0, operand.dimension_size[i] - size_indices[i])`
|
|
|
|
Value start_index =
|
|
|
|
rewriter.create<tensor::ExtractOp>(loc, std::get<0>(en.value()));
|
|
|
|
Value ub = rewriter.createOrFold<memref::DimOp>(loc, adaptor.operand(),
|
|
|
|
en.index());
|
|
|
|
// ClampOp lowering does not support index type, so cast it into integer
|
|
|
|
// type.
|
|
|
|
ub = rewriter.createOrFold<IndexCastOp>(loc, start_index.getType(), ub);
|
|
|
|
ub = rewriter.createOrFold<SubIOp>(
|
|
|
|
loc, ub,
|
|
|
|
rewriter.create<ConstantOp>(
|
|
|
|
loc, rewriter.getIntegerAttr(start_index.getType(), size)));
|
|
|
|
// TODO(hanchung): This is a workaround to use the method because only
|
|
|
|
// lmhlo version is defined. The implementation in
|
|
|
|
// map_lmhlo_to_scalar_op.h requires to pass a mhlo op. It will convert it
|
|
|
|
// to an lmhlo op and call the lmhlo implementation.
|
|
|
|
start_index = lmhlo::HloOpToStdScalarOp::map<lmhlo::ClampOp>(
|
2021-04-29 17:26:25 +08:00
|
|
|
loc, start_index.getType(),
|
|
|
|
ArrayRef<Type>{start_index.getType(), start_index.getType(),
|
|
|
|
start_index.getType()},
|
|
|
|
ArrayRef<Value>{zero, start_index, ub}, &rewriter);
|
2021-04-13 01:34:01 +08:00
|
|
|
start_indices.push_back(
|
|
|
|
rewriter.create<IndexCastOp>(loc, index_type, start_index)
|
|
|
|
.getResult());
|
|
|
|
}
|
|
|
|
|
|
|
|
int64_t rank = arg_type.getRank();
|
|
|
|
SmallVector<OpFoldResult, 3> strides(rank, rewriter.getI64IntegerAttr(1));
|
|
|
|
|
|
|
|
rewriter.replaceOpWithNewOp<SubTensorOp>(
|
|
|
|
dynamic_slice_op, dynamic_slice_op.getType().cast<RankedTensorType>(),
|
|
|
|
adaptor.operand(), start_indices, sizes, strides);
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-01-12 02:33:14 +08:00
|
|
|
enum class DotOperationType {
|
|
|
|
kVectorDot = 0,
|
|
|
|
kMatrixVector = 1,
|
|
|
|
kMatrixMatrix = 2,
|
|
|
|
kUnsupported = 3
|
|
|
|
};
|
|
|
|
|
|
|
|
DotOperationType GetDotOperationType(mhlo::DotOp dot_op) {
|
|
|
|
ArrayRef<int64_t> lhs_shape =
|
|
|
|
dot_op.lhs().getType().cast<ShapedType>().getShape();
|
|
|
|
ArrayRef<int64_t> rhs_shape =
|
|
|
|
dot_op.rhs().getType().cast<ShapedType>().getShape();
|
|
|
|
auto shape_matches = [](int64_t a, int64_t b) {
|
|
|
|
return a == ShapedType::kDynamicSize || b == ShapedType::kDynamicSize ||
|
|
|
|
a == b;
|
|
|
|
};
|
|
|
|
if (lhs_shape.size() == 1 && rhs_shape.size() == 1 &&
|
|
|
|
shape_matches(lhs_shape[0], rhs_shape[0])) {
|
|
|
|
return DotOperationType::kVectorDot;
|
|
|
|
}
|
|
|
|
if (lhs_shape.size() == 2 && rhs_shape.size() == 1 &&
|
|
|
|
shape_matches(lhs_shape[1], rhs_shape[0])) {
|
|
|
|
return DotOperationType::kMatrixVector;
|
|
|
|
}
|
|
|
|
if (rhs_shape.size() == 2 && rhs_shape.size() == 2 &&
|
|
|
|
shape_matches(lhs_shape[1], rhs_shape[0])) {
|
|
|
|
return DotOperationType::kMatrixMatrix;
|
|
|
|
}
|
|
|
|
return DotOperationType::kUnsupported;
|
|
|
|
}
|
|
|
|
|
2021-02-04 07:02:21 +08:00
|
|
|
SmallVector<Value, 2> GetDotOpInitTensorDynSizes(OpBuilder& b, Location loc,
|
2021-01-12 02:33:14 +08:00
|
|
|
Value lhs, Value rhs,
|
|
|
|
DotOperationType type) {
|
2021-02-04 07:02:21 +08:00
|
|
|
SmallVector<Value, 2> dyn_shape;
|
2021-01-12 02:33:14 +08:00
|
|
|
switch (type) {
|
|
|
|
case DotOperationType::kMatrixMatrix: {
|
2021-02-04 07:02:21 +08:00
|
|
|
if (lhs.getType().cast<ShapedType>().isDynamicDim(0))
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, lhs, 0));
|
2021-02-04 07:02:21 +08:00
|
|
|
if (rhs.getType().cast<ShapedType>().isDynamicDim(1))
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, rhs, 1));
|
2021-01-12 02:33:14 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case DotOperationType::kMatrixVector: {
|
2021-02-04 07:02:21 +08:00
|
|
|
if (lhs.getType().cast<ShapedType>().isDynamicDim(0))
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, lhs, 0));
|
2021-01-12 02:33:14 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case DotOperationType::kVectorDot:
|
|
|
|
case DotOperationType::kUnsupported:
|
|
|
|
default: {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return dyn_shape;
|
|
|
|
}
|
|
|
|
|
2021-03-03 05:59:50 +08:00
|
|
|
template <DotOperationType op_type, typename LinalgOp>
|
2021-01-12 02:33:14 +08:00
|
|
|
class DotOpOnTensorsConversion : public OpConversionPattern<mhlo::DotOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<mhlo::DotOp>::OpConversionPattern;
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::DotOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
if (!VerifyHloOpBufferOrTensorSemantics</*isLHLO=*/false>(op)) {
|
|
|
|
return failure();
|
|
|
|
}
|
2021-03-03 05:59:50 +08:00
|
|
|
if (GetDotOperationType(op) != op_type) return failure();
|
2021-02-25 01:43:54 +08:00
|
|
|
|
2021-01-12 02:33:14 +08:00
|
|
|
mhlo::DotOp::Adaptor adaptor(args);
|
2021-02-25 01:43:54 +08:00
|
|
|
|
2021-03-03 05:59:50 +08:00
|
|
|
Location loc = op.getLoc();
|
2021-02-25 01:43:54 +08:00
|
|
|
auto output_type = op.getType().cast<ShapedType>();
|
|
|
|
auto output_el_type = output_type.getElementType();
|
|
|
|
auto zero_attr = rewriter.getZeroAttr(output_el_type);
|
2021-01-12 02:33:14 +08:00
|
|
|
Value zero = rewriter.create<ConstantOp>(loc, zero_attr);
|
2021-02-04 07:02:21 +08:00
|
|
|
SmallVector<Value, 2> dyn_shape = GetDotOpInitTensorDynSizes(
|
|
|
|
rewriter, loc, adaptor.lhs(), adaptor.rhs(), op_type);
|
2021-02-25 01:43:54 +08:00
|
|
|
auto init_tensor = GetInitTensor(rewriter, loc, output_type, dyn_shape);
|
2021-02-04 07:02:21 +08:00
|
|
|
Value zero_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, zero).getResult(0);
|
2021-02-25 01:43:54 +08:00
|
|
|
rewriter.replaceOpWithNewOp<LinalgOp>(
|
|
|
|
op, TypeRange{op.getType()}, ValueRange{adaptor.lhs(), adaptor.rhs()},
|
|
|
|
ValueRange{zero_tensor});
|
2021-01-12 02:33:14 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-01-13 14:07:29 +08:00
|
|
|
SmallVector<Value, 8> GetDotGeneralOpInitTensorDynSizes(
|
|
|
|
OpBuilder& b, Location loc, Value lhs, Value rhs, ShapedType result_type) {
|
|
|
|
SmallVector<Value, 8> dyn_shape;
|
|
|
|
if (result_type.isDynamicDim(0))
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, lhs, 0));
|
2021-01-13 14:07:29 +08:00
|
|
|
if (result_type.isDynamicDim(1))
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, lhs, 1));
|
2021-01-13 14:07:29 +08:00
|
|
|
if (result_type.isDynamicDim(2))
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, rhs, 2));
|
2021-01-13 14:07:29 +08:00
|
|
|
return dyn_shape;
|
|
|
|
}
|
|
|
|
|
|
|
|
class DotGeneralOpOnTensorsConversion
|
|
|
|
: public OpConversionPattern<mhlo::DotGeneralOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<mhlo::DotGeneralOp>::OpConversionPattern;
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::DotGeneralOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
if (!VerifyHloOpBufferOrTensorSemantics</*isLHLO=*/false>(op)) {
|
|
|
|
return failure();
|
|
|
|
}
|
2021-02-25 01:43:54 +08:00
|
|
|
|
2021-01-13 14:07:29 +08:00
|
|
|
mhlo::DotDimensionNumbers dim_numbers = op.dot_dimension_numbers();
|
|
|
|
auto lhs_bathcing_dims =
|
|
|
|
Extract1DVector(dim_numbers.lhs_batching_dimensions());
|
|
|
|
auto rhs_bathcing_dims =
|
|
|
|
Extract1DVector(dim_numbers.rhs_batching_dimensions());
|
|
|
|
auto lhs_contracting_dims =
|
|
|
|
Extract1DVector(dim_numbers.lhs_contracting_dimensions());
|
|
|
|
auto rhs_contracting_dims =
|
|
|
|
Extract1DVector(dim_numbers.rhs_contracting_dimensions());
|
|
|
|
if (lhs_bathcing_dims.size() != 1 || lhs_bathcing_dims[0] != 0) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected lhs batching dimensions exactly {0}");
|
|
|
|
}
|
|
|
|
if (rhs_bathcing_dims.size() != 1 || rhs_bathcing_dims[0] != 0) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected rhs batching dimensions exactly {0}");
|
|
|
|
}
|
|
|
|
if (lhs_contracting_dims.size() != 1 || lhs_contracting_dims[0] != 2) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected lhs contracting dimensions exactly {2}");
|
|
|
|
}
|
|
|
|
if (rhs_contracting_dims.size() != 1 || rhs_contracting_dims[0] != 1) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected rhs contracting dimensions exactly {1}");
|
|
|
|
}
|
2021-02-25 01:43:54 +08:00
|
|
|
|
2021-01-13 14:07:29 +08:00
|
|
|
mhlo::DotGeneralOp::Adaptor adaptor(args);
|
2021-02-25 01:43:54 +08:00
|
|
|
|
2021-03-03 05:59:50 +08:00
|
|
|
Location loc = op.getLoc();
|
2021-02-25 01:43:54 +08:00
|
|
|
auto output_type = op.getType().cast<ShapedType>();
|
|
|
|
auto output_el_type = output_type.getElementType();
|
2021-01-13 14:07:29 +08:00
|
|
|
SmallVector<Value, 8> dyn_shape = GetDotGeneralOpInitTensorDynSizes(
|
2021-02-25 01:43:54 +08:00
|
|
|
rewriter, loc, adaptor.lhs(), adaptor.rhs(), output_type);
|
|
|
|
auto zero_attr = rewriter.getZeroAttr(output_el_type);
|
2021-01-13 14:07:29 +08:00
|
|
|
Value zero = rewriter.create<ConstantOp>(loc, zero_attr);
|
2021-02-25 01:43:54 +08:00
|
|
|
auto init_tensor = GetInitTensor(rewriter, loc, output_type, dyn_shape);
|
2021-02-04 07:02:21 +08:00
|
|
|
Value zero_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, zero).getResult(0);
|
2021-03-03 05:59:50 +08:00
|
|
|
Operation* linalg_op = rewriter.create<linalg::BatchMatmulOp>(
|
2021-02-25 01:43:54 +08:00
|
|
|
loc, /*resultTensorTypes=*/TypeRange{op.getType()},
|
2021-01-13 14:07:29 +08:00
|
|
|
/*inputs=*/ValueRange{adaptor.lhs(), adaptor.rhs()},
|
2021-02-04 07:02:21 +08:00
|
|
|
/*outputBuffers=*/ValueRange{zero_tensor});
|
2021-02-25 01:43:54 +08:00
|
|
|
|
|
|
|
rewriter.replaceOp(op, linalg_op->getResults());
|
2021-01-13 14:07:29 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-01-28 21:44:49 +08:00
|
|
|
template <typename OpTy>
|
|
|
|
struct ReduceRegionXLAOpConversion : public OpConversionPattern<OpTy> {
|
|
|
|
using OpConversionPattern<OpTy>::OpConversionPattern;
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
OpTy op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
// Only convert the body of reduction ops to std ops.
|
|
|
|
auto parent_op = op.getOperation()->getParentRegion()->getParentOp();
|
|
|
|
if (!isa<mhlo::ReduceOp, linalg::GenericOp, linalg::IndexedGenericOp>(
|
|
|
|
parent_op)) {
|
|
|
|
return failure();
|
|
|
|
}
|
|
|
|
if (!op.getResult().getType().template isa<TensorType>()) return failure();
|
|
|
|
if (llvm::all_of(args, [](Value arg) {
|
|
|
|
return arg.getType().template isa<TensorType>();
|
|
|
|
})) {
|
|
|
|
return failure();
|
|
|
|
}
|
2021-04-23 00:49:40 +08:00
|
|
|
Value result = lmhlo::HloOpToStdScalarOp::map<OpTy>(
|
|
|
|
op, getElementTypeOrSelf(op.getType()), args, &rewriter);
|
2021-01-28 21:44:49 +08:00
|
|
|
rewriter.replaceOp(op, result);
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
SmallVector<Value, 8> GetReduceOpInitTensorDynSizes(
|
|
|
|
OpBuilder& b, Location loc, Value arg, ShapedType result_type,
|
|
|
|
ArrayRef<int64_t> reduction_dims) {
|
|
|
|
llvm::SmallSetVector<int, 4> s;
|
|
|
|
for (auto dim : reduction_dims) s.insert(dim);
|
|
|
|
|
|
|
|
SmallVector<unsigned, 4> parallel_dims;
|
|
|
|
SmallVector<Value, 8> dyn_shape;
|
|
|
|
int rank = arg.getType().cast<RankedTensorType>().getRank();
|
|
|
|
for (int i = 0, j = 0; i < rank; ++i) {
|
|
|
|
if (s.count(i)) continue;
|
|
|
|
if (!result_type.isDynamicDim(j++)) continue;
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_shape.push_back(b.create<memref::DimOp>(loc, arg, i));
|
2021-01-28 21:44:49 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return dyn_shape;
|
|
|
|
}
|
|
|
|
|
|
|
|
class ReduceRegionReturnOpConversion
|
|
|
|
: public OpConversionPattern<mhlo::ReturnOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<mhlo::ReturnOp>::OpConversionPattern;
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::ReturnOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
rewriter.replaceOpWithNewOp<linalg::YieldOp>(op, args);
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
class ReduceOnTensorsConversion : public OpConversionPattern<mhlo::ReduceOp> {
|
|
|
|
public:
|
|
|
|
using OpConversionPattern<mhlo::ReduceOp>::OpConversionPattern;
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::ReduceOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
Location loc = op.getLoc();
|
|
|
|
mhlo::ReduceOp::Adaptor adaptor(args);
|
2021-04-23 00:49:40 +08:00
|
|
|
|
|
|
|
int num_inputs = static_cast<int>(adaptor.inputs().size());
|
|
|
|
auto src_type = adaptor.inputs()[0].getType().cast<ShapedType>();
|
2021-01-28 21:44:49 +08:00
|
|
|
int src_rank = src_type.getRank();
|
|
|
|
if (!src_rank) {
|
|
|
|
return rewriter.notifyMatchFailure(op, "expects known-rank args");
|
|
|
|
}
|
|
|
|
|
2021-04-23 00:49:40 +08:00
|
|
|
SmallVector<int64_t, 4> reduction_dims = Extract1DVector(op.dimensions());
|
|
|
|
|
|
|
|
SmallVector<Value> inputs, outputs;
|
|
|
|
SmallVector<AffineMap, 3> indexing_maps;
|
|
|
|
for (int i = 0; i < num_inputs; ++i) {
|
|
|
|
Value src = adaptor.inputs()[i];
|
|
|
|
if (src.getType() != src_type) return failure();
|
|
|
|
|
|
|
|
// Check if init_value is constant. If so, inline the value into the
|
|
|
|
// region.
|
|
|
|
Value init_value = adaptor.init_values()[i];
|
|
|
|
Attribute init_const_val = GetInitValueAsConst(init_value);
|
|
|
|
if (init_const_val) {
|
|
|
|
init_value = rewriter.create<ConstantOp>(
|
|
|
|
init_value.getDefiningOp()->getLoc(), init_const_val);
|
|
|
|
} else {
|
|
|
|
init_value = rewriter.create<tensor::ExtractOp>(loc, init_value);
|
|
|
|
}
|
|
|
|
|
|
|
|
inputs.push_back(src);
|
|
|
|
auto result_type = op.getResult(i).getType().cast<ShapedType>();
|
|
|
|
SmallVector<Value, 8> dyn_shape = GetReduceOpInitTensorDynSizes(
|
|
|
|
rewriter, loc, src, result_type, reduction_dims);
|
|
|
|
auto init_tensor = GetInitTensor(rewriter, loc, result_type, dyn_shape);
|
|
|
|
Value filled_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, init_value)
|
|
|
|
.result();
|
|
|
|
outputs.push_back(filled_tensor);
|
2021-01-28 21:44:49 +08:00
|
|
|
}
|
|
|
|
|
2021-04-23 00:49:40 +08:00
|
|
|
// Prepare indexing maps for linalg generic op. The elements are for src
|
|
|
|
// and dst. Transpose `src` to make the reduction loops be the innermost,
|
2021-01-28 21:44:49 +08:00
|
|
|
// because it's easier to fully utilize processors.
|
2021-04-23 00:49:40 +08:00
|
|
|
indexing_maps.append(
|
|
|
|
num_inputs, GetTransposeMapForReduction(rewriter.getContext(), src_rank,
|
|
|
|
reduction_dims));
|
2021-01-28 21:44:49 +08:00
|
|
|
|
|
|
|
// The indexing map of `dst` should drop the reduction loops. Since the
|
|
|
|
// reduction loops now are all in the innermost, drops
|
2021-04-23 00:49:40 +08:00
|
|
|
// `reduction_dims.size()` dimensions. We don't need an inverse
|
|
|
|
// permutation here because they are the same.
|
2021-01-28 21:44:49 +08:00
|
|
|
SmallVector<AffineExpr, 4> exprs;
|
|
|
|
for (int i = 0, e = src_rank - reduction_dims.size(); i < e; ++i)
|
|
|
|
exprs.push_back(rewriter.getAffineDimExpr(i));
|
2021-04-23 00:49:40 +08:00
|
|
|
indexing_maps.append(num_inputs,
|
|
|
|
AffineMap::get(src_rank, /*symbolCount=*/0, exprs,
|
|
|
|
rewriter.getContext()));
|
2021-01-28 21:44:49 +08:00
|
|
|
|
|
|
|
auto linalg_op = rewriter.create<linalg::GenericOp>(
|
|
|
|
loc, /*resultTensorTypes=*/op.getResultTypes(), inputs,
|
2021-04-23 00:49:40 +08:00
|
|
|
/*outputBuffers=*/ValueRange{outputs}, indexing_maps,
|
2021-01-28 21:44:49 +08:00
|
|
|
GetParallelAndReductionIterators(src_rank, reduction_dims.size()));
|
|
|
|
|
|
|
|
// Convert the signature of the body. The reduce op region apply function
|
|
|
|
// has a signature (lhs, rhs) -> output, all of the same tensor type t.
|
|
|
|
// This is converted to a function with the same signature but with
|
|
|
|
// element types. E.g., "(tensor<f32>, tensor<f32>) -> tensor<f32>" will
|
|
|
|
// be converted to "(f32, f32, f32)".
|
|
|
|
Region& region = linalg_op.region();
|
|
|
|
rewriter.inlineRegionBefore(op.body(), region, region.end());
|
2021-04-23 00:49:40 +08:00
|
|
|
TypeConverter::SignatureConversion signature_converter(num_inputs * 2);
|
|
|
|
for (int i = 0; i < num_inputs * 2; ++i)
|
|
|
|
signature_converter.addInputs(i, src_type.getElementType());
|
|
|
|
rewriter.applySignatureConversion(®ion, signature_converter);
|
2021-01-28 21:44:49 +08:00
|
|
|
rewriter.replaceOp(op, linalg_op.getResults());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-02-25 18:59:55 +08:00
|
|
|
/// Converts mhlo.pad operation to linalg.pad_tensor op.
|
|
|
|
struct PadOpOnTensorsConversion : public OpConversionPattern<mhlo::PadOp> {
|
|
|
|
using OpConversionPattern<mhlo::PadOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::PadOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const override {
|
|
|
|
mhlo::PadOp::Adaptor adaptor(args);
|
|
|
|
if (llvm::any_of(
|
|
|
|
op.interior_padding().getValues<APInt>(),
|
|
|
|
[](const APInt& int_val) { return int_val.getZExtValue() != 0; })) {
|
|
|
|
return rewriter.notifyMatchFailure(op, "expected no interior padding");
|
|
|
|
}
|
|
|
|
|
|
|
|
auto loc = op.getLoc();
|
|
|
|
Value padding_val =
|
|
|
|
rewriter.createOrFold<tensor::ExtractOp>(loc, adaptor.padding_value());
|
|
|
|
|
|
|
|
const auto& edge_padding_low = op.edge_padding_low();
|
|
|
|
const auto& edge_padding_high = op.edge_padding_high();
|
|
|
|
SmallVector<OpFoldResult, 4> low, high;
|
|
|
|
for (auto it : llvm::zip(edge_padding_low, edge_padding_high)) {
|
|
|
|
low.push_back(rewriter.createOrFold<ConstantIndexOp>(
|
|
|
|
loc, std::get<0>(it).getZExtValue()));
|
|
|
|
high.push_back(rewriter.createOrFold<ConstantIndexOp>(
|
|
|
|
loc, std::get<1>(it).getZExtValue()));
|
|
|
|
}
|
|
|
|
Type result_type = op.getResult().getType();
|
|
|
|
auto pad_tensor_op = linalg::PadTensorOp::createPadScalarOp(
|
|
|
|
result_type, adaptor.operand(), padding_val, low, high, loc, rewriter);
|
|
|
|
rewriter.replaceOp(op, pad_tensor_op.getResult());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-02-25 21:58:12 +08:00
|
|
|
/// Converts mhlo.conv operation to linalg named op. This only covers normal
|
|
|
|
/// convolution cases. The op must have canonical dimension numbers. Depthwise
|
|
|
|
/// convolution and pointwise convolution are not handled in the conversion.
|
|
|
|
struct NormalConvOpOnTensorsConversion
|
|
|
|
: public OpConversionPattern<mhlo::ConvOp> {
|
|
|
|
using OpConversionPattern<mhlo::ConvOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::ConvOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const override {
|
|
|
|
if (!HasCanonicalDimensionNumbers(op.dimension_numbers())) return failure();
|
|
|
|
if (op.feature_group_count() != 1u) return failure();
|
|
|
|
|
|
|
|
mhlo::ConvOp::Adaptor adaptor(args);
|
|
|
|
Location loc = op.getLoc();
|
|
|
|
Value input = adaptor.lhs();
|
|
|
|
Value filter = adaptor.rhs();
|
|
|
|
auto result_type = op.getResult().getType().cast<ShapedType>();
|
|
|
|
int64_t rank = result_type.getRank();
|
|
|
|
|
|
|
|
// Check if padding is zero.
|
|
|
|
DenseIntElementsAttr padding = op.paddingAttr();
|
2021-03-12 00:40:31 +08:00
|
|
|
if (padding && !isSplatValue(*op.padding(), 0)) {
|
2021-02-25 21:58:12 +08:00
|
|
|
return rewriter.notifyMatchFailure(op, "expected no padding");
|
|
|
|
}
|
|
|
|
|
|
|
|
// The output shape is N spatial_dims F.
|
|
|
|
SmallVector<Value, 8> dyn_sizes;
|
2021-02-27 01:33:21 +08:00
|
|
|
if (result_type.isDynamicDim(0)) {
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_sizes.push_back(rewriter.create<memref::DimOp>(loc, input, 0));
|
2021-02-27 01:33:21 +08:00
|
|
|
}
|
|
|
|
for (int64_t i = 1, e = rank - 1; i < e; ++i) {
|
|
|
|
if (result_type.isDynamicDim(i)) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected output spatial dims to be static shapes");
|
|
|
|
}
|
2021-02-25 21:58:12 +08:00
|
|
|
}
|
|
|
|
if (result_type.isDynamicDim(rank - 1)) {
|
2021-03-17 04:31:59 +08:00
|
|
|
dyn_sizes.push_back(
|
|
|
|
rewriter.create<memref::DimOp>(loc, filter, rank - 1));
|
2021-02-25 21:58:12 +08:00
|
|
|
}
|
|
|
|
Value init_tensor = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, dyn_sizes, result_type.getShape(), result_type.getElementType());
|
|
|
|
auto zero_attr = rewriter.getZeroAttr(result_type.getElementType());
|
|
|
|
Value zero = rewriter.create<ConstantOp>(loc, zero_attr);
|
|
|
|
Value zero_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, zero).getResult(0);
|
|
|
|
linalg::LinalgOp res;
|
|
|
|
Attribute strides = op.window_stridesAttr();
|
|
|
|
// TODO(ataei): Only support dilated kernel right now. We need to consider
|
|
|
|
// input dilation for deconvolution cases.
|
|
|
|
Attribute dilations = op.rhs_dilationAttr();
|
|
|
|
switch (rank) {
|
|
|
|
case 3: {
|
|
|
|
res = rewriter.create<linalg::ConvInputNWCFilterWCFOp>(
|
|
|
|
loc, result_type, ValueRange{input, filter},
|
|
|
|
ValueRange{zero_tensor}, dilations, strides);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case 4: {
|
|
|
|
res = rewriter.create<linalg::ConvInputNHWCFilterHWCFOp>(
|
|
|
|
loc, result_type, ValueRange{input, filter},
|
|
|
|
ValueRange{zero_tensor}, dilations, strides);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case 5: {
|
|
|
|
res = rewriter.create<linalg::ConvInputNDHWCFilterDHWCFOp>(
|
|
|
|
loc, result_type, ValueRange{input, filter},
|
|
|
|
ValueRange{zero_tensor}, dilations, strides);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
default:
|
|
|
|
return rewriter.notifyMatchFailure(op, "expected 1/2/3D conv op");
|
|
|
|
}
|
|
|
|
rewriter.replaceOp(op, res.getOperation()->getResults());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-12 00:40:31 +08:00
|
|
|
/// Converts mhlo.convolution operation to
|
|
|
|
/// linalg.depthwise_conv_2d_input_nhwc_filter_hwcf op or
|
|
|
|
/// depthwise_conv_2d_input_nhwc_filter_hwc op.
|
|
|
|
struct DepthwiseConvOpOnTensorsConversion
|
|
|
|
: public OpConversionPattern<mhlo::ConvOp> {
|
|
|
|
using OpConversionPattern<mhlo::ConvOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::ConvOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const override {
|
|
|
|
if (op.batch_group_count() != 1) return failure();
|
|
|
|
|
|
|
|
if (op.padding() && !isSplatValue(*op.padding(), 0)) {
|
|
|
|
return rewriter.notifyMatchFailure(op,
|
|
|
|
"non-zero padding unsupported yet");
|
|
|
|
}
|
|
|
|
|
|
|
|
if ((op.lhs_dilation() && !isSplatValue(*op.lhs_dilation(), 1)) ||
|
|
|
|
(op.rhs_dilation() && !isSplatValue(*op.rhs_dilation(), 1))) {
|
|
|
|
return rewriter.notifyMatchFailure(op,
|
|
|
|
"non-one dialation unsupported yet");
|
|
|
|
}
|
|
|
|
|
|
|
|
if (const mhlo::ConvDimensionNumbers& dimension_numbers =
|
|
|
|
op.dimension_numbers()) {
|
|
|
|
// Make sure that this is 2-D convolution.
|
|
|
|
const auto spatial_rank =
|
|
|
|
llvm::size(dimension_numbers.input_spatial_dimensions());
|
|
|
|
if (spatial_rank != 2) {
|
|
|
|
return rewriter.notifyMatchFailure(op,
|
|
|
|
"only support 2-D cases for now");
|
|
|
|
}
|
|
|
|
|
|
|
|
// Make sure that this is depthwise convolution.
|
|
|
|
int64_t input_feature_dim =
|
|
|
|
dimension_numbers.input_feature_dimension().getInt();
|
|
|
|
int64_t input_feature_count =
|
|
|
|
op.lhs().getType().cast<ShapedType>().getDimSize(input_feature_dim);
|
|
|
|
if (op.feature_group_count() != input_feature_count) {
|
|
|
|
return rewriter.notifyMatchFailure(op, "not depth-wise convolution");
|
|
|
|
}
|
|
|
|
|
|
|
|
// Make sure that this convolution has a canonical form.
|
|
|
|
if (!HasCanonicalDimensionNumbers(dimension_numbers)) {
|
|
|
|
return rewriter.notifyMatchFailure(op, "does not have canonical form");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
DenseIntElementsAttr window_strides;
|
|
|
|
if (op.window_strides()) {
|
|
|
|
window_strides = op.window_strides().getValue();
|
|
|
|
} else {
|
|
|
|
window_strides = rewriter.getI64VectorAttr({1, 1});
|
|
|
|
}
|
|
|
|
|
|
|
|
mhlo::ConvOp::Adaptor adaptor(args);
|
|
|
|
Location loc = op.getLoc();
|
|
|
|
Value input = adaptor.lhs();
|
|
|
|
Value filter = adaptor.rhs();
|
|
|
|
auto result_type = op.getResult().getType().cast<RankedTensorType>();
|
|
|
|
if (!result_type.hasStaticShape()) {
|
|
|
|
return rewriter.notifyMatchFailure(op,
|
|
|
|
"expected output has static shapes");
|
|
|
|
}
|
|
|
|
|
|
|
|
auto filter_dims =
|
|
|
|
llvm::to_vector<4>(op.rhs().getType().cast<ShapedType>().getShape());
|
|
|
|
|
|
|
|
auto get_indices_vector = [](int start, int end) {
|
|
|
|
return llvm::to_vector<2>(llvm::seq<int64_t>(start, end));
|
|
|
|
};
|
|
|
|
|
|
|
|
if (filter_dims[2] * filter_dims[3] != op.feature_group_count()) {
|
|
|
|
// For cases where channel multiplier != 1
|
|
|
|
auto output_dims = result_type.getShape();
|
|
|
|
auto channel_multiplier = filter_dims[3];
|
|
|
|
SmallVector<int64_t> reshaped_output_dims;
|
|
|
|
reshaped_output_dims.assign(output_dims.begin(), output_dims.end());
|
|
|
|
reshaped_output_dims.push_back(channel_multiplier);
|
|
|
|
reshaped_output_dims[3] /= channel_multiplier;
|
|
|
|
|
|
|
|
Value init_tensor = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, reshaped_output_dims, result_type.getElementType());
|
|
|
|
auto zero_attr = rewriter.getZeroAttr(result_type.getElementType());
|
|
|
|
Value zero = rewriter.create<ConstantOp>(loc, zero_attr);
|
|
|
|
Value zero_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, zero).getResult(0);
|
|
|
|
|
|
|
|
auto reshaped_output_type = RankedTensorType::get(
|
|
|
|
reshaped_output_dims, result_type.getElementType());
|
|
|
|
auto conv = rewriter.create<linalg::DepthwiseConvInputNHWCFilterHWCFOp>(
|
|
|
|
op.getLoc(), reshaped_output_type, ValueRange{input, filter},
|
|
|
|
ValueRange{zero_tensor}, window_strides);
|
|
|
|
|
|
|
|
// Create a Linalg reshape op that converts the output from 5 dimensions
|
|
|
|
// into 4 dimensions (by collapsing the last two dimensions). This is
|
|
|
|
// needed because linalg.depthwise_conv_2d_input_nhwc_filter_hwcf returns
|
|
|
|
// 5 dimensions for the output.
|
|
|
|
SmallVector<linalg::ReassociationIndices, 4> collapsed_dim_list = {
|
|
|
|
get_indices_vector(0, 1), get_indices_vector(1, 2),
|
|
|
|
get_indices_vector(2, 3), get_indices_vector(3, 5)};
|
|
|
|
rewriter.replaceOpWithNewOp<linalg::TensorReshapeOp>(
|
|
|
|
op, result_type, conv.getResult(0), collapsed_dim_list);
|
|
|
|
} else {
|
|
|
|
// For cases where channel multiplier == 1
|
|
|
|
Value init_tensor = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, result_type.getShape(), result_type.getElementType());
|
|
|
|
auto zero_attr = rewriter.getZeroAttr(result_type.getElementType());
|
|
|
|
Value zero = rewriter.create<ConstantOp>(loc, zero_attr);
|
|
|
|
Value zero_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, zero).getResult(0);
|
|
|
|
|
|
|
|
// Create a Linalg reshape op that converts the filter from 4 dimensions
|
|
|
|
// into 3 dimensions (by droping the unit dimension). This is needed
|
|
|
|
// because linalg.depthwise_conv_2d_input_nhwc_filter_hwc expects 3
|
|
|
|
// dimensions for the filter.
|
|
|
|
|
|
|
|
filter_dims[2] = static_cast<int64_t>(op.feature_group_count());
|
|
|
|
filter_dims.pop_back();
|
|
|
|
|
|
|
|
RankedTensorType filter_shape =
|
|
|
|
RankedTensorType::get(filter_dims, op.getType().getElementType());
|
|
|
|
|
|
|
|
SmallVector<linalg::ReassociationIndices, 4> collapsed_dim_list = {
|
|
|
|
get_indices_vector(0, 1), get_indices_vector(1, 2),
|
|
|
|
get_indices_vector(2, 4)};
|
|
|
|
|
|
|
|
Value reshaped_filter = rewriter.create<linalg::TensorReshapeOp>(
|
|
|
|
loc, filter_shape, filter, collapsed_dim_list);
|
|
|
|
|
|
|
|
rewriter.replaceOpWithNewOp<linalg::DepthwiseConvInputNHWCFilterHWCOp>(
|
|
|
|
op, result_type, ValueRange{input, reshaped_filter},
|
|
|
|
ValueRange{zero_tensor}, window_strides);
|
|
|
|
}
|
|
|
|
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-12 01:40:31 +08:00
|
|
|
struct ReduceWindowOpOnTensorsConversion
|
|
|
|
: public OpConversionPattern<mhlo::ReduceWindowOp> {
|
|
|
|
using OpConversionPattern<mhlo::ReduceWindowOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
/// mhlo.reduce_window is mapped to a linalg.pooling operation. The type of
|
|
|
|
/// the pooling is determined based on the body of the reduce window
|
|
|
|
/// operation. This class enumerates the different variants.
|
|
|
|
enum class PoolingType {
|
2021-04-02 01:23:35 +08:00
|
|
|
kInvalid,
|
2021-03-12 01:40:31 +08:00
|
|
|
kMin,
|
|
|
|
kMax,
|
|
|
|
kAdd,
|
|
|
|
};
|
|
|
|
|
2021-04-02 01:23:35 +08:00
|
|
|
static PoolingType getPoolingType(mhlo::ReduceWindowOp reduce_op,
|
|
|
|
int result_index) {
|
|
|
|
if (Operation* op = reduce_op.getReductionOp(result_index)) {
|
|
|
|
if (isa<mhlo::MinOp>(*op)) return PoolingType::kMin;
|
|
|
|
if (isa<mhlo::MaxOp>(*op)) return PoolingType::kMax;
|
|
|
|
if (isa<mhlo::AddOp>(*op)) return PoolingType::kAdd;
|
|
|
|
}
|
|
|
|
return PoolingType::kInvalid;
|
2021-03-12 01:40:31 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::ReduceWindowOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const override {
|
|
|
|
auto loc = op.getLoc();
|
2021-04-02 01:23:35 +08:00
|
|
|
int rank = op.getResultTypes()[0].cast<ShapedType>().getRank();
|
|
|
|
if (rank != 4) {
|
2021-03-12 01:40:31 +08:00
|
|
|
return rewriter.notifyMatchFailure(op, "expected NHWC pooling-based op");
|
|
|
|
}
|
|
|
|
|
2021-04-07 20:45:07 +08:00
|
|
|
if (op.padding() && !isSplatValue(*op.padding(), 0)) {
|
|
|
|
return rewriter.notifyMatchFailure(op, "require paddings are all zero");
|
|
|
|
}
|
|
|
|
|
2021-04-02 01:23:35 +08:00
|
|
|
SmallVector<int64_t, 2> shapes;
|
2021-03-12 01:40:31 +08:00
|
|
|
shapes.push_back(op.window_dimensions().getValue<int64_t>(1));
|
|
|
|
shapes.push_back(op.window_dimensions().getValue<int64_t>(2));
|
|
|
|
|
|
|
|
if (op.window_strides() &&
|
|
|
|
(op.window_strides().getValue().getValue<int64_t>(0) != 1 ||
|
|
|
|
op.window_strides().getValue().getValue<int64_t>(3) != 1)) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected window_strides to be [1,x,y,1]");
|
|
|
|
}
|
|
|
|
if (op.window_dimensions() &&
|
|
|
|
(op.window_dimensions().getValue<int64_t>(0) != 1 ||
|
|
|
|
op.window_dimensions().getValue<int64_t>(3) != 1)) {
|
|
|
|
return rewriter.notifyMatchFailure(
|
|
|
|
op, "expected window_dimensions to be [1,x,y,1]");
|
|
|
|
}
|
|
|
|
|
|
|
|
Attribute strides;
|
|
|
|
if (op.window_stridesAttr()) {
|
|
|
|
strides = rewriter.getI64VectorAttr(
|
|
|
|
{op.window_strides().getValue().getValue<int64_t>(1),
|
|
|
|
op.window_strides().getValue().getValue<int64_t>(2)});
|
|
|
|
} else {
|
|
|
|
strides = rewriter.getI64VectorAttr({1, 1});
|
|
|
|
}
|
|
|
|
Attribute dilations;
|
|
|
|
if (op.window_dilations()) {
|
|
|
|
dilations = rewriter.getI64VectorAttr(
|
|
|
|
{op.window_dilations().getValue().getValue<int64_t>(1),
|
|
|
|
op.window_dilations().getValue().getValue<int64_t>(2)});
|
|
|
|
} else {
|
|
|
|
dilations = rewriter.getI64VectorAttr({1, 1});
|
|
|
|
}
|
|
|
|
|
2021-04-02 01:23:35 +08:00
|
|
|
SmallVector<Value> pooling_ops;
|
|
|
|
|
|
|
|
ArrayRef<Value> inputs = args.take_front(op.inputs().size());
|
|
|
|
ArrayRef<Value> init_values = args.drop_front(op.inputs().size());
|
|
|
|
for (auto it : llvm::zip(op.getResults(), inputs, init_values)) {
|
|
|
|
OpResult result = std::get<0>(it);
|
|
|
|
Value input = std::get<1>(it);
|
|
|
|
Value init_value = std::get<2>(it);
|
|
|
|
auto result_type = result.getType().cast<ShapedType>();
|
|
|
|
if (!input.getType().cast<ShapedType>().getElementType().isF32()) {
|
|
|
|
return rewriter.notifyMatchFailure(op,
|
|
|
|
"expected element type to be f32");
|
2021-03-12 01:40:31 +08:00
|
|
|
}
|
2021-04-02 01:23:35 +08:00
|
|
|
|
|
|
|
// Create a fake window dimension.
|
|
|
|
auto fake_window_dims = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, shapes, result_type.getElementType());
|
|
|
|
Value init_tensor = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, result_type.getShape(), result_type.getElementType());
|
|
|
|
init_value = rewriter.create<tensor::ExtractOp>(loc, init_value);
|
|
|
|
Value filled_init_tensor =
|
|
|
|
rewriter.create<linalg::FillOp>(loc, init_tensor, init_value)
|
|
|
|
.getResult(0);
|
|
|
|
auto create_op = [&](auto* type_ptr) -> linalg::LinalgOp {
|
|
|
|
return cast<linalg::LinalgOp>(
|
|
|
|
rewriter
|
|
|
|
.create<std::remove_pointer_t<decltype(type_ptr)>>(
|
|
|
|
loc, ArrayRef<Type>{result_type},
|
|
|
|
ValueRange{args[0], fake_window_dims.getResult()},
|
|
|
|
filled_init_tensor, dilations, strides)
|
|
|
|
.getOperation());
|
|
|
|
};
|
|
|
|
linalg::LinalgOp pooling_op;
|
|
|
|
PoolingType pooling_type = getPoolingType(op, result.getResultNumber());
|
|
|
|
switch (pooling_type) {
|
|
|
|
case PoolingType::kMin: {
|
|
|
|
pooling_op =
|
2021-04-07 03:39:46 +08:00
|
|
|
create_op(static_cast<linalg::PoolingNHWCMinFOp*>(nullptr));
|
2021-04-02 01:23:35 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case PoolingType::kMax: {
|
|
|
|
pooling_op =
|
2021-04-07 03:39:46 +08:00
|
|
|
create_op(static_cast<linalg::PoolingNHWCMaxFOp*>(nullptr));
|
2021-04-02 01:23:35 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case PoolingType::kAdd: {
|
|
|
|
pooling_op =
|
2021-04-07 03:39:46 +08:00
|
|
|
create_op(static_cast<linalg::PoolingNHWCSumFOp*>(nullptr));
|
2021-04-02 01:23:35 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case PoolingType::kInvalid:
|
|
|
|
return rewriter.notifyMatchFailure(op, "unknown reduction operation");
|
2021-03-12 01:40:31 +08:00
|
|
|
}
|
2021-04-02 01:23:35 +08:00
|
|
|
pooling_ops.push_back(pooling_op->getResult(0));
|
2021-03-12 01:40:31 +08:00
|
|
|
}
|
2021-04-02 01:23:35 +08:00
|
|
|
rewriter.replaceOp(op, pooling_ops);
|
2021-03-12 01:40:31 +08:00
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-17 21:31:58 +08:00
|
|
|
/// Converts xla-hlo.torch_index_select op to a linalg.indexed_generic op.
|
|
|
|
struct TorchIndexSelectOpOnTensorsConversion
|
|
|
|
: public OpConversionPattern<mhlo::TorchIndexSelectOp> {
|
|
|
|
using OpConversionPattern<mhlo::TorchIndexSelectOp>::OpConversionPattern;
|
|
|
|
|
|
|
|
LogicalResult matchAndRewrite(
|
|
|
|
mhlo::TorchIndexSelectOp op, ArrayRef<Value> args,
|
|
|
|
ConversionPatternRewriter& rewriter) const final {
|
|
|
|
mhlo::TorchIndexSelectOp::Adaptor adaptor(args);
|
|
|
|
int axis = static_cast<int>(op.dim());
|
|
|
|
int batch = static_cast<int>(op.batch_dims());
|
|
|
|
auto index_shaped_type = adaptor.index().getType().cast<ShapedType>();
|
|
|
|
int num_indices = static_cast<int>(index_shaped_type.getRank());
|
|
|
|
auto input_shaped_type = adaptor.input().getType().cast<ShapedType>();
|
|
|
|
if (axis < 0) axis += static_cast<int>(input_shaped_type.getRank());
|
|
|
|
if (batch < 0) batch += num_indices;
|
|
|
|
|
|
|
|
Location loc = op.getLoc();
|
|
|
|
auto result_type = op.getResult().getType().cast<ShapedType>();
|
|
|
|
int rank = static_cast<int>(result_type.getRank());
|
|
|
|
|
|
|
|
SmallVector<AffineMap, 2> indexing_maps;
|
|
|
|
SmallVector<AffineExpr, 4> exprs;
|
|
|
|
for (int i = 0; i < batch; ++i) {
|
|
|
|
exprs.push_back(rewriter.getAffineDimExpr(i));
|
|
|
|
}
|
|
|
|
for (int i = 0, e = num_indices - batch; i < e; ++i) {
|
|
|
|
exprs.push_back(rewriter.getAffineDimExpr(axis + i));
|
|
|
|
}
|
|
|
|
indexing_maps.emplace_back(
|
|
|
|
AffineMap::get(rank, /*symbolCount=*/0, exprs, rewriter.getContext()));
|
|
|
|
indexing_maps.emplace_back(rewriter.getMultiDimIdentityMap(rank));
|
|
|
|
|
|
|
|
// The output shape is
|
|
|
|
// `params[:axis] + indices[batch_dims:] + params[axis + 1:]`
|
|
|
|
SmallVector<Value, 4> dyn_sizes;
|
|
|
|
for (int i = 0; i < rank; ++i) {
|
|
|
|
if (!result_type.isDynamicDim(i)) continue;
|
|
|
|
if (i < axis) {
|
|
|
|
dyn_sizes.push_back(
|
|
|
|
rewriter.create<memref::DimOp>(loc, adaptor.input(), i));
|
|
|
|
} else if (i < (axis + num_indices - batch)) {
|
|
|
|
int idx = i - axis + batch;
|
|
|
|
dyn_sizes.push_back(
|
|
|
|
rewriter.create<memref::DimOp>(loc, adaptor.index(), idx));
|
|
|
|
} else {
|
|
|
|
int idx = i - (axis + num_indices - batch) + axis + 1;
|
|
|
|
dyn_sizes.push_back(
|
|
|
|
rewriter.create<memref::DimOp>(loc, adaptor.input(), idx));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
Value init_op = rewriter.create<linalg::InitTensorOp>(
|
|
|
|
loc, dyn_sizes, result_type.getShape(), result_type.getElementType());
|
|
|
|
auto linalg_op = rewriter.create<linalg::IndexedGenericOp>(
|
|
|
|
loc, /*resultTensors=*/ArrayRef<Type>{result_type},
|
|
|
|
/*inputs=*/adaptor.index(),
|
|
|
|
/*outputs=*/init_op, indexing_maps, GetNParallelLoopsAttrs(rank));
|
|
|
|
|
|
|
|
SmallVector<Type, 4> body_arg_types;
|
|
|
|
SmallVector<Value, 2> linalg_op_args = {adaptor.index()};
|
|
|
|
// Add a block to the region.
|
|
|
|
auto* region = &linalg_op.region();
|
|
|
|
auto* block = rewriter.createBlock(region, region->end());
|
|
|
|
body_arg_types.append(rank, rewriter.getIndexType());
|
|
|
|
for (auto block_args : linalg_op_args) {
|
|
|
|
body_arg_types.push_back(
|
|
|
|
block_args.getType().cast<ShapedType>().getElementType());
|
|
|
|
}
|
|
|
|
block->addArguments(body_arg_types);
|
|
|
|
block->addArguments(result_type.getElementType());
|
|
|
|
OpBuilder::InsertionGuard guard(rewriter);
|
|
|
|
rewriter.setInsertionPointToEnd(block);
|
|
|
|
|
|
|
|
SmallVector<Value, 4> indices;
|
|
|
|
Value casted_value = rewriter.create<IndexCastOp>(
|
|
|
|
loc, block->getArgument(rank), rewriter.getIndexType());
|
|
|
|
for (int i = 0; i < axis; ++i) {
|
|
|
|
indices.push_back(block->getArgument(i));
|
|
|
|
}
|
|
|
|
indices.push_back(casted_value);
|
|
|
|
for (int i = axis + num_indices - batch; i < rank; ++i) {
|
|
|
|
indices.push_back(block->getArgument(i));
|
|
|
|
}
|
|
|
|
|
|
|
|
Value res =
|
|
|
|
rewriter.create<tensor::ExtractOp>(loc, adaptor.input(), indices);
|
|
|
|
rewriter.create<linalg::YieldOp>(loc, res);
|
|
|
|
|
|
|
|
rewriter.replaceOp(op, linalg_op.getResults());
|
|
|
|
return success();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
void populateLHLOToLinalgConversionPattern(MLIRContext* context,
|
2021-04-29 17:26:25 +08:00
|
|
|
TypeConverter& typeConverter,
|
2020-07-07 04:57:00 +08:00
|
|
|
OwningRewritePatternList* patterns) {
|
|
|
|
// clang-format off
|
2020-07-09 01:05:32 +08:00
|
|
|
patterns->insert<BroadcastConverter<lmhlo::BroadcastOp>,
|
2020-12-10 15:24:23 +08:00
|
|
|
ConstConverter<lmhlo::ConstOp>,
|
2020-07-07 04:57:00 +08:00
|
|
|
ConvToLinalgConverter,
|
2020-07-24 00:18:01 +08:00
|
|
|
IotaConverter<lmhlo::IotaOp>,
|
2020-07-07 04:57:00 +08:00
|
|
|
LhloBroadcastInDimConverter,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::AbsOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::AddOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::AndOp>,
|
2020-10-05 20:06:35 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::Atan2Op>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::CeilOp>,
|
2021-01-15 22:44:28 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::ClampOp>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::CompareOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::ComplexOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::ConvertOp>,
|
2020-07-07 04:57:00 +08:00
|
|
|
// TODO(ataei): Remove this pattern, CopyOp is folded away.
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::CopyOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::CosOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::DivOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::ExpOp>,
|
2021-02-18 20:53:34 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::Expm1Op>,
|
2020-08-31 23:15:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::FloorOp>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::ImagOp>,
|
2020-12-08 22:38:26 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::IsFiniteOp>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::LogOp>,
|
2021-01-22 19:02:13 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::LogisticOp>,
|
2021-01-13 21:36:19 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::Log1pOp>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::MaxOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::MinOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::MulOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::NegOp>,
|
2020-09-29 20:58:52 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::NotOp>,
|
2020-12-08 22:38:26 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::OrOp>,
|
2020-12-22 07:26:38 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::PowOp>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::RealOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::RemOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::RsqrtOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::SelectOp>,
|
2020-12-08 05:01:25 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::ShiftLeftOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::ShiftRightArithmeticOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::ShiftRightLogicalOp>,
|
2020-07-09 01:05:32 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::SignOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::SinOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::SqrtOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::SubOp>,
|
|
|
|
PointwiseToLinalgConverter<lmhlo::TanhOp>,
|
2020-12-08 22:38:26 +08:00
|
|
|
PointwiseToLinalgConverter<lmhlo::XorOp>,
|
2020-10-29 07:37:38 +08:00
|
|
|
ReduceConverter,
|
2020-07-09 01:05:32 +08:00
|
|
|
ReshapeOpConverter<lmhlo::ReshapeOp>,
|
|
|
|
ReverseConverter<lmhlo::ReverseOp>,
|
|
|
|
ScalarPointwiseToStandardConverter<lmhlo::AddOp>,
|
2020-10-29 07:37:38 +08:00
|
|
|
ScalarPointwiseToStandardConverter<lmhlo::MaxOp>,
|
2021-02-25 01:04:32 +08:00
|
|
|
SliceConverter<lmhlo::SliceOp>,
|
2020-09-05 06:37:02 +08:00
|
|
|
TransposeConverter<lmhlo::TransposeOp>
|
2021-04-29 17:26:25 +08:00
|
|
|
>(typeConverter, context);
|
2020-07-07 04:57:00 +08:00
|
|
|
// clang-format on
|
|
|
|
}
|
|
|
|
|
2021-04-29 17:26:25 +08:00
|
|
|
// Converter that turns signed/unsigned integers types into signless types.
|
|
|
|
class RemoveSignTypeConverter : public TypeConverter {
|
|
|
|
public:
|
|
|
|
RemoveSignTypeConverter() {
|
|
|
|
addConversion([](Type type) { return type; });
|
|
|
|
|
|
|
|
addConversion(convertInteger);
|
|
|
|
addConversion(convertShapedType);
|
|
|
|
|
|
|
|
addArgumentMaterialization(materializeCastFromIllegal);
|
|
|
|
addSourceMaterialization(materializeCastToIllegal);
|
|
|
|
addTargetMaterialization(materializeCastFromIllegal);
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
static Type convertInteger(IntegerType int_type) {
|
|
|
|
return IntegerType::get(int_type.getContext(),
|
|
|
|
int_type.getIntOrFloatBitWidth());
|
|
|
|
}
|
|
|
|
|
|
|
|
static Type convertShapedType(ShapedType shaped_type) {
|
|
|
|
if (auto int_type = shaped_type.getElementType().dyn_cast<IntegerType>())
|
|
|
|
return shaped_type.clone(convertInteger(int_type));
|
|
|
|
return shaped_type;
|
|
|
|
}
|
|
|
|
|
|
|
|
static llvm::Optional<Value> materializeCastFromIllegal(OpBuilder& builder,
|
|
|
|
Type type,
|
|
|
|
ValueRange inputs,
|
|
|
|
Location loc) {
|
|
|
|
Type from_type = getElementTypeOrSelf(inputs[0].getType());
|
|
|
|
Type to_type = getElementTypeOrSelf(type);
|
|
|
|
if ((!from_type.isSignedInteger() && !from_type.isUnsignedInteger()) ||
|
|
|
|
!to_type.isSignlessInteger())
|
|
|
|
return llvm::None;
|
|
|
|
// Use unrealized conversion casts to do signful->signless conversions.
|
|
|
|
return builder.create<UnrealizedConversionCastOp>(loc, type, inputs[0])
|
|
|
|
->getResult(0);
|
|
|
|
}
|
|
|
|
|
|
|
|
static llvm::Optional<Value> materializeCastToIllegal(OpBuilder& builder,
|
|
|
|
Type type,
|
|
|
|
ValueRange inputs,
|
|
|
|
Location loc) {
|
|
|
|
Type from_type = getElementTypeOrSelf(inputs[0].getType());
|
|
|
|
Type to_type = getElementTypeOrSelf(type);
|
|
|
|
if (!from_type.isSignlessInteger() ||
|
|
|
|
(!to_type.isSignedInteger() && !to_type.isUnsignedInteger()))
|
|
|
|
return llvm::None;
|
|
|
|
// Use unrealized conversion casts to do signless->signful conversions.
|
|
|
|
return builder.create<UnrealizedConversionCastOp>(loc, type, inputs[0])
|
|
|
|
->getResult(0);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
// Converts LHLO ops to Linalg generic.
|
2020-07-09 01:05:32 +08:00
|
|
|
// Sample result for lmhlo::AddOp.
|
2020-07-07 04:57:00 +08:00
|
|
|
//
|
2020-07-09 01:05:32 +08:00
|
|
|
// "lmhlo.add"(%arg1, %arg2, %out) :
|
2020-07-07 04:57:00 +08:00
|
|
|
// (memref<2x2xf32>, memref<2x2xf32>, memref<2x2xf32>) -> ()
|
|
|
|
//
|
|
|
|
// will be converted to
|
|
|
|
//
|
|
|
|
// #map0 = (d0, d1) -> (d0, d1)
|
|
|
|
// "linalg.generic"(%arg1, %arg2, %out) ( {
|
|
|
|
// ^bb0(%arg4: f32, %arg5: f32):
|
|
|
|
// %0 = addf %arg4, %arg5 : f32
|
|
|
|
// "linalg.yield"(%0) : (f32) -> ()
|
|
|
|
// }) {
|
|
|
|
// indexing_maps = [#map0, #map0, #map0],
|
|
|
|
// iterator_types = ["parallel", "parallel"],
|
|
|
|
// } : (memref<2x2xf32>, memref<2x2xf32>, memref<2x2xf32>) -> ()
|
2020-07-29 07:12:08 +08:00
|
|
|
struct LhloLegalizeToLinalgPass
|
|
|
|
: public PassWrapper<LhloLegalizeToLinalgPass, FunctionPass> {
|
2020-08-26 11:30:05 +08:00
|
|
|
void getDependentDialects(DialectRegistry& registry) const override {
|
2021-03-30 16:06:12 +08:00
|
|
|
registry
|
|
|
|
.insert<AffineDialect, complex::ComplexDialect, linalg::LinalgDialect,
|
|
|
|
math::MathDialect, memref::MemRefDialect>();
|
2020-08-26 11:30:05 +08:00
|
|
|
}
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
void runOnFunction() override {
|
2021-03-24 03:18:57 +08:00
|
|
|
OwningRewritePatternList patterns(&getContext());
|
2020-07-07 04:57:00 +08:00
|
|
|
ConversionTarget target(getContext());
|
2021-01-21 17:21:23 +08:00
|
|
|
target.addLegalDialect<complex::ComplexDialect, linalg::LinalgDialect,
|
2021-03-17 04:31:59 +08:00
|
|
|
math::MathDialect, memref::MemRefDialect,
|
|
|
|
StandardOpsDialect, AffineDialect>();
|
2021-04-29 17:26:25 +08:00
|
|
|
target.addLegalOp<UnrealizedConversionCastOp>();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2021-04-29 17:26:25 +08:00
|
|
|
RemoveSignTypeConverter type_converter;
|
2020-07-07 04:57:00 +08:00
|
|
|
auto func = getFunction();
|
2021-04-29 17:26:25 +08:00
|
|
|
populateLHLOToLinalgConversionPattern(func.getContext(), type_converter,
|
|
|
|
&patterns);
|
2020-10-27 21:55:28 +08:00
|
|
|
if (failed(applyPartialConversion(func, target, std::move(patterns)))) {
|
2020-07-07 04:57:00 +08:00
|
|
|
signalPassFailure();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-07-29 07:12:08 +08:00
|
|
|
struct HloLegalizeToLinalgPass
|
|
|
|
: public PassWrapper<HloLegalizeToLinalgPass, FunctionPass> {
|
2020-08-26 11:30:05 +08:00
|
|
|
void getDependentDialects(DialectRegistry& registry) const override {
|
2021-03-30 16:06:12 +08:00
|
|
|
registry
|
|
|
|
.insert<linalg::LinalgDialect, scf::SCFDialect, complex::ComplexDialect,
|
|
|
|
math::MathDialect, memref::MemRefDialect>();
|
2020-08-26 11:30:05 +08:00
|
|
|
}
|
|
|
|
|
2020-07-07 04:57:00 +08:00
|
|
|
void runOnFunction() override {
|
2021-04-29 17:26:25 +08:00
|
|
|
MLIRContext& ctx = getContext();
|
|
|
|
OwningRewritePatternList patterns(&ctx);
|
|
|
|
ConversionTarget target(ctx);
|
2021-01-21 17:21:23 +08:00
|
|
|
target.addLegalDialect<complex::ComplexDialect, linalg::LinalgDialect,
|
2021-02-13 00:30:51 +08:00
|
|
|
math::MathDialect, StandardOpsDialect,
|
|
|
|
tensor::TensorDialect, scf::SCFDialect>();
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2021-03-17 04:31:59 +08:00
|
|
|
// TODO: DimOp shouldn't be in MemRefDialect
|
|
|
|
target.addLegalOp<memref::DimOp>();
|
2021-04-29 17:26:25 +08:00
|
|
|
target.addLegalOp<UnrealizedConversionCastOp>();
|
2021-03-17 04:31:59 +08:00
|
|
|
|
2021-04-29 17:26:25 +08:00
|
|
|
RemoveSignTypeConverter type_converter;
|
2020-07-07 04:57:00 +08:00
|
|
|
auto func = getFunction();
|
2021-04-29 17:26:25 +08:00
|
|
|
mhlo::populateHLOToLinalgConversionPattern(&ctx, type_converter, &patterns);
|
2020-10-27 21:55:28 +08:00
|
|
|
if (failed(applyPartialConversion(func, target, std::move(patterns)))) {
|
2020-07-07 04:57:00 +08:00
|
|
|
signalPassFailure();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
} // namespace
|
|
|
|
|
2020-07-09 01:05:32 +08:00
|
|
|
namespace lmhlo {
|
2020-07-07 04:57:00 +08:00
|
|
|
std::unique_ptr<OperationPass<FuncOp>> createLegalizeLhloToLinalgPass() {
|
2020-07-29 07:12:08 +08:00
|
|
|
return std::make_unique<LhloLegalizeToLinalgPass>();
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2020-07-09 01:05:32 +08:00
|
|
|
} // namespace lmhlo
|
2020-07-07 04:57:00 +08:00
|
|
|
|
2020-07-07 12:51:24 +08:00
|
|
|
namespace mhlo {
|
2020-07-07 04:57:00 +08:00
|
|
|
|
|
|
|
void populateHLOToLinalgConversionPattern(MLIRContext* context,
|
2021-04-29 17:26:25 +08:00
|
|
|
TypeConverter& type_converter,
|
2020-07-07 04:57:00 +08:00
|
|
|
OwningRewritePatternList* patterns) {
|
2021-02-25 18:59:55 +08:00
|
|
|
// clang-format off
|
2021-01-28 21:44:49 +08:00
|
|
|
patterns->insert<
|
2021-04-22 01:00:12 +08:00
|
|
|
BroadcastConverter<mhlo::BroadcastOp, false>, ConcatenateConverter,
|
2021-01-28 21:44:49 +08:00
|
|
|
ConstConverter<mhlo::ConstOp>, HloDynamicBroadcastInDimConverter,
|
|
|
|
HloBroadcastInDimConverter, IotaConverter<mhlo::IotaOp, false>,
|
2021-03-12 00:30:08 +08:00
|
|
|
IotaConverter<mhlo::DynamicIotaOp, false>,
|
2021-01-28 21:44:49 +08:00
|
|
|
PointwiseToLinalgConverter<mhlo::AbsOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::AddOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::AndOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::Atan2Op, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::CeilOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ClampOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::CompareOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ComplexOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ConvertOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::CopyOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::CosOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::DivOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ExpOp, false>,
|
2021-02-18 20:53:34 +08:00
|
|
|
PointwiseToLinalgConverter<mhlo::Expm1Op, false>,
|
2021-01-28 21:44:49 +08:00
|
|
|
PointwiseToLinalgConverter<mhlo::FloorOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ImagOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::IsFiniteOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::LogOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::LogisticOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::Log1pOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::MaxOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::MinOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::MulOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::NegOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::NotOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::OrOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::PowOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::RealOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::RemOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::RsqrtOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::SelectOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ShiftLeftOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ShiftRightArithmeticOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::ShiftRightLogicalOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::SignOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::SinOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::SqrtOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::SubOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::TanhOp, false>,
|
|
|
|
PointwiseToLinalgConverter<mhlo::XorOp, false>,
|
|
|
|
ReshapeOpConverter<mhlo::ReshapeOp, false>,
|
|
|
|
ReverseConverter<mhlo::ReverseOp, false>,
|
2021-02-25 01:04:32 +08:00
|
|
|
SliceConverter<mhlo::SliceOp, false>,
|
2021-04-13 01:34:01 +08:00
|
|
|
DynamicSliceConverter,
|
2021-02-25 01:43:54 +08:00
|
|
|
TransposeConverter<mhlo::TransposeOp, false>,
|
2021-03-03 05:59:50 +08:00
|
|
|
DotOpOnTensorsConversion<DotOperationType::kMatrixMatrix,
|
2021-02-25 01:43:54 +08:00
|
|
|
linalg::MatmulOp>,
|
2021-03-03 05:59:50 +08:00
|
|
|
DotOpOnTensorsConversion<DotOperationType::kMatrixVector,
|
2021-02-25 01:43:54 +08:00
|
|
|
linalg::MatvecOp>,
|
2021-03-03 05:59:50 +08:00
|
|
|
DotOpOnTensorsConversion<DotOperationType::kVectorDot, linalg::DotOp>,
|
|
|
|
DotGeneralOpOnTensorsConversion,
|
2021-02-25 21:58:12 +08:00
|
|
|
NormalConvOpOnTensorsConversion,
|
2021-03-12 00:40:31 +08:00
|
|
|
DepthwiseConvOpOnTensorsConversion,
|
2021-02-25 18:59:55 +08:00
|
|
|
ReduceOnTensorsConversion,
|
2021-03-12 01:40:31 +08:00
|
|
|
ReduceWindowOpOnTensorsConversion,
|
2021-03-17 21:31:58 +08:00
|
|
|
TorchIndexSelectOpOnTensorsConversion,
|
2021-04-29 17:26:25 +08:00
|
|
|
PadOpOnTensorsConversion>(type_converter, context);
|
2021-02-25 18:59:55 +08:00
|
|
|
// clang-format on
|
2021-01-28 21:44:49 +08:00
|
|
|
patterns->insert<ReduceRegionXLAOpConversion<mhlo::AddOp>,
|
|
|
|
ReduceRegionXLAOpConversion<mhlo::MinOp>,
|
|
|
|
ReduceRegionXLAOpConversion<mhlo::MaxOp>,
|
2021-04-09 22:07:41 +08:00
|
|
|
ReduceRegionXLAOpConversion<mhlo::AndOp>,
|
|
|
|
ReduceRegionXLAOpConversion<mhlo::OrOp>,
|
2021-04-23 00:49:40 +08:00
|
|
|
ReduceRegionXLAOpConversion<mhlo::SelectOp>,
|
|
|
|
ReduceRegionXLAOpConversion<mhlo::CompareOp>,
|
2021-01-28 21:44:49 +08:00
|
|
|
ReduceRegionReturnOpConversion>(context);
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
std::unique_ptr<OperationPass<FuncOp>> createLegalizeHloToLinalgPass() {
|
2020-07-29 07:12:08 +08:00
|
|
|
return std::make_unique<HloLegalizeToLinalgPass>();
|
2020-07-07 04:57:00 +08:00
|
|
|
}
|
2021-04-29 17:26:25 +08:00
|
|
|
|
|
|
|
std::unique_ptr<TypeConverter> createHloToLinalgSignedIntegerConverter() {
|
|
|
|
return std::make_unique<RemoveSignTypeConverter>();
|
|
|
|
}
|
|
|
|
|
2020-07-07 12:51:24 +08:00
|
|
|
} // namespace mhlo
|
2020-07-07 04:57:00 +08:00
|
|
|
} // namespace mlir
|