MLIR  21.0.0git
LowerGpuOpsToROCDLOps.cpp
Go to the documentation of this file.
1 //===- LowerGpuOpsToROCDLOps.cpp - MLIR GPU to ROCDL lowering passes ------===//
2 //
3 // Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
4 // See https://llvm.org/LICENSE.txt for license information.
5 // SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
6 //
7 //===----------------------------------------------------------------------===//
8 //
9 // This file implements a pass to generate ROCDLIR operations for higher-level
10 // GPU operations.
11 //
12 //===----------------------------------------------------------------------===//
13 
16 #include "mlir/Pass/Pass.h"
17 #include "mlir/Pass/PassManager.h"
18 #include "mlir/Transforms/Passes.h"
19 
40 #include "mlir/Pass/Pass.h"
43 #include "llvm/Support/FormatVariadic.h"
44 
45 #include "../GPUCommon/GPUOpsLowering.h"
46 #include "../GPUCommon/IndexIntrinsicsOpLowering.h"
47 
48 namespace mlir {
49 #define GEN_PASS_DEF_CONVERTGPUOPSTOROCDLOPS
50 #include "mlir/Conversion/Passes.h.inc"
51 } // namespace mlir
52 
53 using namespace mlir;
54 
55 /// Returns true if the given `gpu.func` can be safely called using the bare
56 /// pointer calling convention.
57 static bool canBeCalledWithBarePointers(gpu::GPUFuncOp func) {
58  bool canBeBare = true;
59  for (Type type : func.getArgumentTypes())
60  if (auto memrefTy = dyn_cast<BaseMemRefType>(type))
61  canBeBare &= LLVMTypeConverter::canConvertToBarePtr(memrefTy);
62  return canBeBare;
63 }
64 
66  const unsigned indexBitwidth) {
67  auto int32Type = IntegerType::get(rewriter.getContext(), 32);
68  Value zero = rewriter.create<arith::ConstantIntOp>(loc, 0, 32);
69  Value minus1 = rewriter.create<arith::ConstantIntOp>(loc, -1, 32);
70  Value mbcntLo = rewriter.create<ROCDL::MbcntLoOp>(loc, int32Type,
71  ValueRange{minus1, zero});
72  Value laneId = rewriter.create<ROCDL::MbcntHiOp>(loc, int32Type,
73  ValueRange{minus1, mbcntLo});
74  return laneId;
75 }
76 static constexpr StringLiteral amdgcnDataLayout =
77  "e-p:64:64-p1:64:64-p2:32:32-p3:32:32-p4:64:64-p5:32:32-p6:32:32"
78  "-p7:160:256:256:32-p8:128:128-p9:192:256:256:32-i64:64-v16:16-v24:32-v32:"
79  "32-v48:64-v96:128-v192:256-v256:256-v512:512-v1024:1024-v2048:2048-n32:"
80  "64-S32-A5-G1-ni:7:8:9";
81 
82 namespace {
83 struct GPULaneIdOpToROCDL : ConvertOpToLLVMPattern<gpu::LaneIdOp> {
85 
86  LogicalResult
87  matchAndRewrite(gpu::LaneIdOp op, gpu::LaneIdOp::Adaptor adaptor,
88  ConversionPatternRewriter &rewriter) const override {
89  auto loc = op->getLoc();
90  MLIRContext *context = rewriter.getContext();
91  // convert to: %mlo = call @llvm.amdgcn.mbcnt.lo(-1, 0)
92  // followed by: %lid = call @llvm.amdgcn.mbcnt.hi(-1, %mlo)
93 
94  Type intTy = IntegerType::get(context, 32);
95  Value zero = rewriter.create<arith::ConstantIntOp>(loc, 0, 32);
96  Value minus1 = rewriter.create<arith::ConstantIntOp>(loc, -1, 32);
97  Value mbcntLo =
98  rewriter.create<ROCDL::MbcntLoOp>(loc, intTy, ValueRange{minus1, zero});
99  Value laneId = rewriter.create<ROCDL::MbcntHiOp>(
100  loc, intTy, ValueRange{minus1, mbcntLo});
101  // Truncate or extend the result depending on the index bitwidth specified
102  // by the LLVMTypeConverter options.
103  const unsigned indexBitwidth = getTypeConverter()->getIndexTypeBitwidth();
104  if (indexBitwidth > 32) {
105  laneId = rewriter.create<LLVM::SExtOp>(
106  loc, IntegerType::get(context, indexBitwidth), laneId);
107  } else if (indexBitwidth < 32) {
108  laneId = rewriter.create<LLVM::TruncOp>(
109  loc, IntegerType::get(context, indexBitwidth), laneId);
110  }
111  rewriter.replaceOp(op, {laneId});
112  return success();
113  }
114 };
115 
116 struct GPUShuffleOpLowering : public ConvertOpToLLVMPattern<gpu::ShuffleOp> {
118 
119  /// Lowers a shuffle to the corresponding ROCDL ops.
120  ///
121  /// Use the `width` argument to see if src lane is participating.
122  /// If not the dstLane would be itself.
123  ///
124  /// Shuffle with DS Bpermute:
125  /// let shflMode = [xor, up, down, idx]
126  /// let width = 32(usually warpsize), step = [1, 2, 4, 8, 16, ... , width].
127  /// 1. curLaneId = using mbcnt.lo + mbcnt.hi
128  /// 2. widthOrZeroIfOutside = (curLaneId + width) & -width
129  /// 3. dstLane = shflMode(curLaneId, step)
130  /// 4. isActiveSrcLane = dstLane < isActiveSrcLane
131  /// 5. dstLane = isActiveSrcLane ? dstLane : curLaneId
132  /// 6. dwordAlignedDstLane = dstLane * 4 or dstLane << 2.
133  /// 7. bpermute(dwordAlignedDstLane, shfl_value).
134  ///
135  LogicalResult
136  matchAndRewrite(gpu::ShuffleOp op, OpAdaptor adaptor,
137  ConversionPatternRewriter &rewriter) const override {
138  Location loc = op->getLoc();
139  // TODO: Add support for non 32-bit shuffle values.
140  if (adaptor.getValue().getType().getIntOrFloatBitWidth() != 32)
141  return failure();
142  const unsigned indexBitwidth = getTypeConverter()->getIndexTypeBitwidth();
143  Value srcLaneId = getLaneId(rewriter, loc, indexBitwidth);
144 
145  auto int32Type = IntegerType::get(rewriter.getContext(), 32);
146  Value width = adaptor.getWidth();
147  Value zero = rewriter.create<LLVM::ConstantOp>(loc, int32Type, 0);
148  Value negwidth = rewriter.create<LLVM::SubOp>(loc, int32Type, zero, width);
149  Value add = rewriter.create<LLVM::AddOp>(loc, int32Type, srcLaneId, width);
150  Value widthOrZeroIfOutside =
151  rewriter.create<LLVM::AndOp>(loc, int32Type, add, negwidth);
152  Value dstLane;
153  // TODO: Add support for gpu::ShuffleMode::UP and gpu::ShuffleMode::DOWN.
154  // TODO: Use ds_swizzle for XOR when step/offsets are constants for better
155  // perf.
156  switch (op.getMode()) {
157  case gpu::ShuffleMode::DOWN:
158  dstLane = rewriter.create<LLVM::AddOp>(loc, int32Type, srcLaneId,
159  adaptor.getOffset());
160  break;
161  case gpu::ShuffleMode::XOR:
162  dstLane = rewriter.create<LLVM::XOrOp>(loc, int32Type, srcLaneId,
163  adaptor.getOffset());
164  break;
165  case gpu::ShuffleMode::IDX:
166  dstLane = adaptor.getOffset();
167  break;
168  default:
169  return failure();
170  }
171  Value isActiveSrcLane = rewriter.create<LLVM::ICmpOp>(
172  loc, LLVM::ICmpPredicate::slt, dstLane, widthOrZeroIfOutside);
173  Value selectDstLane = rewriter.create<LLVM::SelectOp>(loc, isActiveSrcLane,
174  dstLane, srcLaneId);
175  Value two = rewriter.create<LLVM::ConstantOp>(loc, int32Type, 2);
176  Value dwordAlignedDstLane =
177  rewriter.create<LLVM::ShlOp>(loc, int32Type, selectDstLane, two);
178  Value initShflValue = adaptor.getValue();
179  if (adaptor.getValue().getType().isF32()) {
180  initShflValue =
181  rewriter.create<LLVM::BitcastOp>(loc, int32Type, initShflValue);
182  }
183  Value shflValue = rewriter.create<ROCDL::DsBpermuteOp>(
184  loc, int32Type, dwordAlignedDstLane, initShflValue);
185  if (adaptor.getValue().getType().isF32()) {
186  shflValue = rewriter.create<LLVM::BitcastOp>(
187  loc, adaptor.getValue().getType(), shflValue);
188  }
189  rewriter.replaceOp(op, {shflValue, isActiveSrcLane});
190  return success();
191  }
192 };
193 
194 /// Import the GPU Ops to ROCDL Patterns.
195 #include "GPUToROCDL.cpp.inc"
196 
197 // A pass that replaces all occurrences of GPU device operations with their
198 // corresponding ROCDL equivalent.
199 //
200 // This pass only handles device code and is not meant to be run on GPU host
201 // code.
202 struct LowerGpuOpsToROCDLOpsPass final
203  : public impl::ConvertGpuOpsToROCDLOpsBase<LowerGpuOpsToROCDLOpsPass> {
204  LowerGpuOpsToROCDLOpsPass() = default;
205  LowerGpuOpsToROCDLOpsPass(const std::string &chipset, unsigned indexBitwidth,
206  bool useBarePtrCallConv,
207  gpu::amd::Runtime runtime) {
208  if (this->chipset.getNumOccurrences() == 0)
209  this->chipset = chipset;
210  if (this->indexBitwidth.getNumOccurrences() == 0)
211  this->indexBitwidth = indexBitwidth;
212  if (this->useBarePtrCallConv.getNumOccurrences() == 0)
213  this->useBarePtrCallConv = useBarePtrCallConv;
214  if (this->runtime.getNumOccurrences() == 0)
215  this->runtime = runtime;
216  }
217 
218  void getDependentDialects(DialectRegistry &registry) const override {
219  Base::getDependentDialects(registry);
221  }
222 
223  void runOnOperation() override {
224  gpu::GPUModuleOp m = getOperation();
225  MLIRContext *ctx = m.getContext();
226 
227  auto llvmDataLayout = m->getAttrOfType<StringAttr>(
228  LLVM::LLVMDialect::getDataLayoutAttrName());
229  if (!llvmDataLayout) {
230  llvmDataLayout = StringAttr::get(ctx, amdgcnDataLayout);
231  m->setAttr(LLVM::LLVMDialect::getDataLayoutAttrName(), llvmDataLayout);
232  }
233  // Request C wrapper emission.
234  for (auto func : m.getOps<func::FuncOp>()) {
235  func->setAttr(LLVM::LLVMDialect::getEmitCWrapperAttrName(),
236  UnitAttr::get(ctx));
237  }
238 
239  FailureOr<amdgpu::Chipset> maybeChipset = amdgpu::Chipset::parse(chipset);
240  if (failed(maybeChipset)) {
241  emitError(UnknownLoc::get(ctx), "Invalid chipset name: " + chipset);
242  return signalPassFailure();
243  }
244 
245  /// Customize the bitwidth used for the device side index computations.
247  ctx, DataLayout(cast<DataLayoutOpInterface>(m.getOperation())));
248  options.dataLayout = llvm::DataLayout(llvmDataLayout.getValue());
249  if (indexBitwidth != kDeriveIndexBitwidthFromDataLayout)
250  options.overrideIndexBitwidth(indexBitwidth);
251 
252  if (useBarePtrCallConv) {
253  options.useBarePtrCallConv = true;
254  WalkResult canUseBarePointers =
255  m.walk([](gpu::GPUFuncOp func) -> WalkResult {
256  if (canBeCalledWithBarePointers(func))
257  return WalkResult::advance();
258  return WalkResult::interrupt();
259  });
260  if (canUseBarePointers.wasInterrupted()) {
262  "bare pointer calling convention requires all memrefs to "
263  "have static shape and use the identity map");
264  return signalPassFailure();
265  }
266  }
267 
268  // Apply in-dialect lowering. In-dialect lowering will replace
269  // ops which need to be lowered further, which is not supported by a
270  // single conversion pass.
271  {
275  (void)applyPatternsGreedily(m, std::move(patterns));
276  }
277 
278  LLVMTypeConverter converter(ctx, options);
280  converter, [](gpu::AddressSpace space) {
281  switch (space) {
282  case gpu::AddressSpace::Global:
283  return 1;
284  case gpu::AddressSpace::Workgroup:
285  return 3;
286  case gpu::AddressSpace::Private:
287  return 5;
288  }
289  llvm_unreachable("unknown address space enum value");
290  return 0;
291  });
292 
293  RewritePatternSet llvmPatterns(ctx);
295 
296  llvm::SmallDenseSet<StringRef> allowedDialectsSet(allowedDialects.begin(),
297  allowedDialects.end());
298  for (Dialect *dialect : ctx->getLoadedDialects()) {
299  bool allowed = allowedDialectsSet.contains(dialect->getNamespace());
300  // Empty `allowedDialectsSet` means all dialects are allowed.
301  if (!allowedDialectsSet.empty() && !allowed)
302  continue;
303 
304  auto iface = dyn_cast<ConvertToLLVMPatternInterface>(dialect);
305  if (!iface) {
306  // Error out if dialect was explicily specified but doesn't implement
307  // conversion interface.
308  if (allowed) {
309  m.emitError()
310  << "dialect does not implement ConvertToLLVMPatternInterface: "
311  << dialect->getNamespace();
312  return signalPassFailure();
313  }
314  continue;
315  }
316 
317  iface->populateConvertToLLVMConversionPatterns(target, converter,
318  llvmPatterns);
319  }
320 
321  populateAMDGPUToROCDLConversionPatterns(converter, llvmPatterns,
322  *maybeChipset);
323  populateGpuToROCDLConversionPatterns(converter, llvmPatterns, runtime);
325  if (failed(applyPartialConversion(m, target, std::move(llvmPatterns))))
326  signalPassFailure();
327  auto *rocdlDialect = getContext().getLoadedDialect<ROCDL::ROCDLDialect>();
328  auto reqdWorkGroupSizeAttrHelper =
329  rocdlDialect->getReqdWorkGroupSizeAttrHelper();
330  auto flatWorkGroupSizeAttrHelper =
331  rocdlDialect->getFlatWorkGroupSizeAttrHelper();
332  // Manually rewrite known block size attributes so the LLVMIR translation
333  // infrastructure can pick them up.
334  m.walk([&](LLVM::LLVMFuncOp op) {
335  if (reqdWorkGroupSizeAttrHelper.isAttrPresent(op)) {
336  auto blockSizes = reqdWorkGroupSizeAttrHelper.getAttr(op);
337  // Also set up the rocdl.flat_work_group_size attribute to prevent
338  // conflicting metadata.
339  uint32_t flatSize = 1;
340  for (uint32_t size : blockSizes.asArrayRef()) {
341  flatSize *= size;
342  }
343  StringAttr flatSizeAttr =
344  StringAttr::get(ctx, Twine(flatSize) + "," + Twine(flatSize));
345  flatWorkGroupSizeAttrHelper.setAttr(op, flatSizeAttr);
346  }
347  });
348  }
349 };
350 
351 } // namespace
352 
354  target.addIllegalOp<func::FuncOp>();
355  target.addLegalDialect<::mlir::LLVM::LLVMDialect>();
356  target.addLegalDialect<ROCDL::ROCDLDialect>();
357  target.addIllegalDialect<gpu::GPUDialect>();
358  target.addIllegalOp<LLVM::CosOp, LLVM::ExpOp, LLVM::Exp2Op, LLVM::FCeilOp,
359  LLVM::FFloorOp, LLVM::FRemOp, LLVM::LogOp, LLVM::Log10Op,
360  LLVM::Log2Op, LLVM::PowOp, LLVM::SinOp>();
361  // These ops are legal for f32 type.
362  target.addDynamicallyLegalOp<LLVM::ExpOp, LLVM::LogOp>([](Operation *op) {
363  return any_of(op->getOperandTypes(), llvm::IsaPred<Float32Type>);
364  });
365  // TODO: Remove once we support replacing non-root ops.
366  target.addLegalOp<gpu::YieldOp, gpu::GPUModuleOp>();
367 }
368 
370  const LLVMTypeConverter &converter, RewritePatternSet &patterns,
371  mlir::gpu::amd::Runtime runtime) {
375  auto *rocdlDialect =
376  converter.getContext().getLoadedDialect<ROCDL::ROCDLDialect>();
377  populateWithGenerated(patterns);
378  patterns.add<
379  gpu::index_lowering::OpLowering<gpu::ThreadIdOp, ROCDL::ThreadIdXOp,
380  ROCDL::ThreadIdYOp, ROCDL::ThreadIdZOp>>(
381  converter, IndexKind::Block, IntrType::Id);
383  gpu::BlockIdOp, ROCDL::BlockIdXOp, ROCDL::BlockIdYOp, ROCDL::BlockIdZOp>>(
384  converter, IndexKind::Grid, IntrType::Id);
385  patterns.add<
386  gpu::index_lowering::OpLowering<gpu::BlockDimOp, ROCDL::BlockDimXOp,
387  ROCDL::BlockDimYOp, ROCDL::BlockDimZOp>>(
388  converter, IndexKind::Block, IntrType::Dim);
390  gpu::GridDimOp, ROCDL::GridDimXOp, ROCDL::GridDimYOp, ROCDL::GridDimZOp>>(
391  converter, IndexKind::Grid, IntrType::Dim);
392  patterns.add<GPUReturnOpLowering>(converter);
394  converter,
396  /*allocaAddrSpace=*/ROCDL::ROCDLDialect::kPrivateMemoryAddressSpace,
397  /*workgroupAddrSpace=*/ROCDL::ROCDLDialect::kSharedMemoryAddressSpace,
398  rocdlDialect->getKernelAttrHelper().getName(),
399  rocdlDialect->getReqdWorkGroupSizeAttrHelper().getName()});
400  if (Runtime::HIP == runtime) {
401  patterns.add<GPUPrintfOpToHIPLowering>(converter);
402  } else if (Runtime::OpenCL == runtime) {
403  // Use address space = 4 to match the OpenCL definition of printf()
404  patterns.add<GPUPrintfOpToLLVMCallLowering>(converter, /*addressSpace=*/4);
405  }
406  // TODO: Add alignment for workgroup memory
408 
409  patterns.add<GPUShuffleOpLowering, GPULaneIdOpToROCDL>(converter);
410 
412 }
413 
414 std::unique_ptr<OperationPass<gpu::GPUModuleOp>>
415 mlir::createLowerGpuOpsToROCDLOpsPass(const std::string &chipset,
416  unsigned indexBitwidth,
417  bool useBarePtrCallConv,
418  gpu::amd::Runtime runtime) {
419  return std::make_unique<LowerGpuOpsToROCDLOpsPass>(
420  chipset, indexBitwidth, useBarePtrCallConv, runtime);
421 }
static MLIRContext * getContext(OpFoldResult val)
static bool canBeCalledWithBarePointers(gpu::GPUFuncOp func)
Returns true if the given gpu.func can be safely called using the bare pointer calling convention.
static constexpr StringLiteral amdgcnDataLayout
Value getLaneId(ConversionPatternRewriter &rewriter, Location loc, const unsigned indexBitwidth)
static llvm::ManagedStatic< PassManagerOptions > options
MLIRContext * getContext() const
Definition: Builders.h:56
This class implements a pattern rewriter for use with ConversionPatterns.
void replaceOp(Operation *op, ValueRange newValues) override
Replace the given operation with the new values.
This class describes a specific conversion target.
void addLegalOp(OperationName op)
Register the given operations as legal.
void addLegalDialect(StringRef name, Names... names)
Register the operations of the given dialects as legal.
void addDynamicallyLegalOp(OperationName op, const DynamicLegalityCallbackFn &callback)
Register the given operation as dynamically legal and set the dynamic legalization callback to the on...
void addIllegalDialect(StringRef name, Names... names)
Register the operations of the given dialects as illegal, i.e.
void addIllegalOp(OperationName op)
Register the given operation as illegal, i.e.
Utility class for operation conversions targeting the LLVM dialect that match exactly one source oper...
Definition: Pattern.h:143
The main mechanism for performing data layout queries.
The DialectRegistry maps a dialect namespace to a constructor for the matching dialect.
Dialects are groups of MLIR operations, types and attributes, as well as behavior associated with the...
Definition: Dialect.h:38
Derived class that automatically populates legalization information for different LLVM ops.
Conversion from types to the LLVM IR dialect.
Definition: TypeConverter.h:35
static bool canConvertToBarePtr(BaseMemRefType type)
Check if a memref type can be converted to a bare pointer.
MLIRContext & getContext() const
Returns the MLIR context.
This class defines the main interface for locations in MLIR and acts as a non-nullable wrapper around...
Definition: Location.h:66
Options to control the LLVM lowering.
MLIRContext is the top-level object for a collection of MLIR operations.
Definition: MLIRContext.h:60
Dialect * getLoadedDialect(StringRef name)
Get a registered IR dialect with the given namespace.
std::vector< Dialect * > getLoadedDialects()
Return information about all IR dialects loaded in the context.
Operation * create(const OperationState &state)
Creates an operation given the fields represented as an OperationState.
Definition: Builders.cpp:453
Operation is the basic unit of execution within MLIR.
Definition: Operation.h:88
Instances of the Type class are uniqued, have an immutable identifier and an optional mutable compone...
Definition: Types.h:74
This class provides an abstraction over the different types of ranges over Values.
Definition: ValueRange.h:381
This class represents an instance of an SSA value in the MLIR system, representing a computable value...
Definition: Value.h:96
A utility result that is used to signal how to proceed with an ongoing walk:
Definition: Visitors.h:33
static WalkResult advance()
Definition: Visitors.h:51
bool wasInterrupted() const
Returns true if the walk was interrupted.
Definition: Visitors.h:55
static WalkResult interrupt()
Definition: Visitors.h:50
void populateExpandBFloat16Patterns(RewritePatternSet &patterns)
Add patterns to expand Arith bf16 patterns to lower level bitcasts/shifts.
Definition: ExpandOps.cpp:397
Runtime
Potential runtimes for AMD GPU kernels.
Definition: Runtimes.h:15
Include the generated interface declarations.
static constexpr unsigned kDeriveIndexBitwidthFromDataLayout
Value to pass as bitwidth for the index type when the converter is expected to derive the bitwidth fr...
LogicalResult applyPatternsGreedily(Region &region, const FrozenRewritePatternSet &patterns, GreedyRewriteConfig config=GreedyRewriteConfig(), bool *changed=nullptr)
Rewrite ops in the given region, which must be isolated from above, by repeatedly applying the highes...
void populateGpuToROCDLConversionPatterns(const LLVMTypeConverter &converter, RewritePatternSet &patterns, gpu::amd::Runtime runtime)
Collect a set of patterns to convert from the GPU dialect to ROCDL.
void populateGpuRewritePatterns(RewritePatternSet &patterns)
Collect all patterns to rewrite ops within the GPU dialect.
Definition: Passes.h:91
InFlightDiagnostic emitError(Location loc)
Utility method to emit an error message using this location.
void populateAMDGPUToROCDLConversionPatterns(const LLVMTypeConverter &converter, RewritePatternSet &patterns, amdgpu::Chipset chipset)
Note: The ROCDL target does not support the LLVM bfloat type at this time and so this function will a...
void configureGpuToROCDLConversionLegality(ConversionTarget &target)
Configure target to convert from the GPU dialect to ROCDL.
std::unique_ptr< OperationPass< gpu::GPUModuleOp > > createLowerGpuOpsToROCDLOpsPass(const std::string &chipset="gfx900", unsigned indexBitwidth=kDeriveIndexBitwidthFromDataLayout, bool useBarePtrCallConv=false, gpu::amd::Runtime runtime=gpu::amd::Runtime::Unknown)
Creates a pass that lowers GPU dialect operations to ROCDL counterparts.
const FrozenRewritePatternSet & patterns
void registerConvertToLLVMDependentDialectLoading(DialectRegistry &registry)
Register the extension that will load dependent dialects for LLVM conversion.
void populateGpuMemorySpaceAttributeConversions(TypeConverter &typeConverter, const MemorySpaceMapping &mapping)
Populates memory space attribute conversion rules for lowering gpu.address_space to integer values.
auto get(MLIRContext *context, Ts &&...params)
Helper method that injects context only if needed, this helps unify some of the attribute constructio...
LogicalResult applyPartialConversion(ArrayRef< Operation * > ops, const ConversionTarget &target, const FrozenRewritePatternSet &patterns, ConversionConfig config=ConversionConfig())
Below we define several entry points for operation conversion.
void populateMathToROCDLConversionPatterns(const LLVMTypeConverter &converter, RewritePatternSet &patterns)
Populate the given list with patterns that convert from Math to ROCDL calls.
Definition: MathToROCDL.cpp:48
Lowering for gpu.dynamic.shared.memory to LLVM dialect.
The lowering of gpu.printf to a call to HIP hostcalls.
The lowering of gpu.printf to a call to an external printf() function.
static FailureOr< Chipset > parse(StringRef name)
Parses the chipset version string and returns the chipset on success, and failure otherwise.
Definition: Chipset.cpp:14