Files
clang-p2996/flang/lib/Lower/OpenMP/Utils.cpp
agozillon e508bacce4 [Flang][OpenMP] Derived type explicit allocatable member mapping (#113557)
This PR is one of 3 in a PR stack, this is the primary change set which
seeks to extend the current derived type explicit member mapping support
to handle descriptor member mapping at arbitrary levels of nesting. The
PR stack seems to do this reasonably (from testing so far) but as you
can create quite complex mappings with derived types (in particular when
adding allocatable derived types or arrays of allocatable derived types)
I imagine there will be hiccups, which I am more than happy to address.
There will also be further extensions to this work to handle the
implicit auto-magical mapping of descriptor members in derived types and
a few other changes planned for the future (with some ideas on
optimizing things).

The changes in this PR primarily occur in the OpenMP lowering and the
OMPMapInfoFinalization pass.

In the OpenMP lowering several utility functions were added or extended
to support the generation of appropriate intermediate member mappings
which are currently required when the parent (or multiple parents) of a
mapped member are descriptor types. We need to map the entirety of these
types or do a "deep copy" for lack of a better term, where we map both
the base address and the descriptor as without the copying of both of
these we lack the information in the case of the descriptor to access
the member or attach the pointers data to the pointer and in the latter
case we require the base address to map the chunk of data. Currently we
do not segment descriptor based derived types as we do with regular
non-descriptor derived types, we effectively map their entirety in all
cases at the moment, I hope to address this at some point in the future
as it adds a fair bit of a performance penalty to having nestings of
allocatable derived types as an example. The process of mapping all
intermediate descriptor members in a members path only occurs if a
member has an allocatable or object parent in its symbol path or the
member itself is a member or allocatable. This occurs in the
createParentSymAndGenIntermediateMaps function, which will also generate
the appropriate address for the allocatable member within the derived
type to use as a the varPtr field of the map (for intermediate
allocatable maps and final allocatable mappings). In this case it's
necessary as we can't utilise the usual Fortran::lower functionality
such as gatherDataOperandAddrAndBounds without causing issues later in
the lowering due to extra allocas being spawned which seem to affect the
pointer attachment (at least this is my current assumption, it results
in memory access errors on the device due to incorrect map information
generation). This is similar to why we do not use the MLIR value
generated for this and utilise the original symbol provided when mapping
descriptor types external to derived types. Hopefully this can be
rectified in the future so this function can be simplified and more
closely aligned to the other type mappings. We also make use of
fir::CoordinateOp as opposed to the HLFIR version as the HLFIR version
doesn't support the appropriate lowering to FIR necessary at the moment,
we also cannot use a single CoordinateOp (similarly to a single GEP) as
when we index through a descriptor operation (BoxType) we encounter
issues later in the lowering, however in either case we need access to
intermediate descriptors so individual CoordinateOp's aid this
(although, being able to compress them into a smaller amount of
CoordinateOp's may simplify the IR and perhaps result in a better end
product, something to consider for the future).

The other large change area was in the OMPMapInfoFinalization pass,
where the pass had to be extended to support the expansion of box types
(or multiple nestings of box types) within derived types, or box type
derived types. This requires expanding each BoxType mapping from one
into two maps and then modifying all of the existing member indices of
the overarching parent mapping to account for the addition of these new
members alongside adjusting the existing member indices to support the
addition of these new maps which extend the original member indices (as
a base address of a box type is currently considered a member of the box
type at a position of 0 as when lowered to LLVM-IR it's a pointer
contained at this position in the descriptor type, however, this means
extending mapped children of this expanded descriptor type to
additionally incorporate the new member index in the correct location in
its own index list). I believe there is a reasonable amount of comments
that should aid in understanding this better, alongside the test
alterations for the pass.

A subset of the changes were also aimed at making some of the utilities
for packing and unpacking the DenseIntElementsAttr containing the member
indices shareable across the lowering and OMPMapInfoFinalization, this
required moving some functions to the Lower/Support/Utils.h header, and
transforming the lowering structure containing the member index data
into something more similar to the version used in
OMPMapInfoFinalization. There we also some other attempts at tidying
things up in relation to the member index data generation in the
lowering, some of which required creating a logical operator for the
OpenMP ID class so it can be utilised as a map key (it simply utilises
the symbol address for the moment as ordering isn't particularly
important).

Otherwise I have added a set of new tests encompassing some of the
mappings currently supported by this PR (unfortunately as you can have
arbitrary nestings of all shapes and types it's not very feasible to
cover them all).
2024-11-16 12:28:37 +01:00

601 lines
26 KiB
C++

//===-- Utils..cpp ----------------------------------------------*- C++ -*-===//
//
// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
// See https://llvm.org/LICENSE.txt for license information.
// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
//
//===----------------------------------------------------------------------===//
//
// Coding style: https://mlir.llvm.org/getting_started/DeveloperGuide/
//
//===----------------------------------------------------------------------===//
#include "Utils.h"
#include "Clauses.h"
#include <DirectivesCommon.h>
#include <flang/Lower/AbstractConverter.h>
#include <flang/Lower/ConvertType.h>
#include <flang/Lower/PFTBuilder.h>
#include <flang/Optimizer/Builder/FIRBuilder.h>
#include <flang/Optimizer/Builder/Todo.h>
#include <flang/Parser/parse-tree.h>
#include <flang/Parser/tools.h>
#include <flang/Semantics/tools.h>
#include <llvm/Support/CommandLine.h>
#include <iterator>
llvm::cl::opt<bool> treatIndexAsSection(
"openmp-treat-index-as-section",
llvm::cl::desc("In the OpenMP data clauses treat `a(N)` as `a(N:N)`."),
llvm::cl::init(true));
llvm::cl::opt<bool> enableDelayedPrivatization(
"openmp-enable-delayed-privatization",
llvm::cl::desc(
"Emit `[first]private` variables as clauses on the MLIR ops."),
llvm::cl::init(true));
llvm::cl::opt<bool> enableDelayedPrivatizationStaging(
"openmp-enable-delayed-privatization-staging",
llvm::cl::desc("For partially supported constructs, emit `[first]private` "
"variables as clauses on the MLIR ops."),
llvm::cl::init(false));
namespace Fortran {
namespace lower {
namespace omp {
int64_t getCollapseValue(const List<Clause> &clauses) {
auto iter = llvm::find_if(clauses, [](const Clause &clause) {
return clause.id == llvm::omp::Clause::OMPC_collapse;
});
if (iter != clauses.end()) {
const auto &collapse = std::get<clause::Collapse>(iter->u);
return evaluate::ToInt64(collapse.v).value();
}
return 1;
}
void genObjectList(const ObjectList &objects,
lower::AbstractConverter &converter,
llvm::SmallVectorImpl<mlir::Value> &operands) {
for (const Object &object : objects) {
const semantics::Symbol *sym = object.sym();
assert(sym && "Expected Symbol");
if (mlir::Value variable = converter.getSymbolAddress(*sym)) {
operands.push_back(variable);
} else if (const auto *details =
sym->detailsIf<semantics::HostAssocDetails>()) {
operands.push_back(converter.getSymbolAddress(details->symbol()));
converter.copySymbolBinding(details->symbol(), *sym);
}
}
}
mlir::Type getLoopVarType(lower::AbstractConverter &converter,
std::size_t loopVarTypeSize) {
// OpenMP runtime requires 32-bit or 64-bit loop variables.
loopVarTypeSize = loopVarTypeSize * 8;
if (loopVarTypeSize < 32) {
loopVarTypeSize = 32;
} else if (loopVarTypeSize > 64) {
loopVarTypeSize = 64;
mlir::emitWarning(converter.getCurrentLocation(),
"OpenMP loop iteration variable cannot have more than 64 "
"bits size and will be narrowed into 64 bits.");
}
assert((loopVarTypeSize == 32 || loopVarTypeSize == 64) &&
"OpenMP loop iteration variable size must be transformed into 32-bit "
"or 64-bit");
return converter.getFirOpBuilder().getIntegerType(loopVarTypeSize);
}
semantics::Symbol *
getIterationVariableSymbol(const lower::pft::Evaluation &eval) {
return eval.visit(common::visitors{
[&](const parser::DoConstruct &doLoop) {
if (const auto &maybeCtrl = doLoop.GetLoopControl()) {
using LoopControl = parser::LoopControl;
if (auto *bounds = std::get_if<LoopControl::Bounds>(&maybeCtrl->u)) {
static_assert(std::is_same_v<decltype(bounds->name),
parser::Scalar<parser::Name>>);
return bounds->name.thing.symbol;
}
}
return static_cast<semantics::Symbol *>(nullptr);
},
[](auto &&) { return static_cast<semantics::Symbol *>(nullptr); },
});
}
void gatherFuncAndVarSyms(
const ObjectList &objects, mlir::omp::DeclareTargetCaptureClause clause,
llvm::SmallVectorImpl<DeclareTargetCapturePair> &symbolAndClause) {
for (const Object &object : objects)
symbolAndClause.emplace_back(clause, *object.sym());
}
mlir::omp::MapInfoOp
createMapInfoOp(fir::FirOpBuilder &builder, mlir::Location loc,
mlir::Value baseAddr, mlir::Value varPtrPtr,
llvm::StringRef name, llvm::ArrayRef<mlir::Value> bounds,
llvm::ArrayRef<mlir::Value> members,
mlir::ArrayAttr membersIndex, uint64_t mapType,
mlir::omp::VariableCaptureKind mapCaptureType, mlir::Type retTy,
bool partialMap) {
if (auto boxTy = llvm::dyn_cast<fir::BaseBoxType>(baseAddr.getType())) {
baseAddr = builder.create<fir::BoxAddrOp>(loc, baseAddr);
retTy = baseAddr.getType();
}
mlir::TypeAttr varType = mlir::TypeAttr::get(
llvm::cast<mlir::omp::PointerLikeType>(retTy).getElementType());
// For types with unknown extents such as <2x?xi32> we discard the incomplete
// type info and only retain the base type. The correct dimensions are later
// recovered through the bounds info.
if (auto seqType = llvm::dyn_cast<fir::SequenceType>(varType.getValue()))
if (seqType.hasDynamicExtents())
varType = mlir::TypeAttr::get(seqType.getEleTy());
mlir::omp::MapInfoOp op = builder.create<mlir::omp::MapInfoOp>(
loc, retTy, baseAddr, varType, varPtrPtr, members, membersIndex, bounds,
builder.getIntegerAttr(builder.getIntegerType(64, false), mapType),
builder.getAttr<mlir::omp::VariableCaptureKindAttr>(mapCaptureType),
builder.getStringAttr(name), builder.getBoolAttr(partialMap));
return op;
}
// This function gathers the individual omp::Object's that make up a
// larger omp::Object symbol.
//
// For example, provided the larger symbol: "parent%child%member", this
// function breaks it up into its constituent components ("parent",
// "child", "member"), so we can access each individual component and
// introspect details. Important to note is this function breaks it up from
// RHS to LHS ("member" to "parent") and then we reverse it so that the
// returned omp::ObjectList is LHS to RHS, with the "parent" at the
// beginning.
omp::ObjectList gatherObjectsOf(omp::Object derivedTypeMember,
semantics::SemanticsContext &semaCtx) {
omp::ObjectList objList;
std::optional<omp::Object> baseObj = derivedTypeMember;
while (baseObj.has_value()) {
objList.push_back(baseObj.value());
baseObj = getBaseObject(baseObj.value(), semaCtx);
}
return omp::ObjectList{llvm::reverse(objList)};
}
// This function generates a series of indices from a provided omp::Object,
// that devolves to an ArrayRef symbol, e.g. "array(2,3,4)", this function
// would generate a series of indices of "[1][2][3]" for the above example,
// offsetting by -1 to account for the non-zero fortran indexes.
//
// These indices can then be provided to a coordinate operation or other
// GEP-like operation to access the relevant positional member of the
// array.
//
// It is of note that the function only supports subscript integers currently
// and not Triplets i.e. Array(1:2:3).
static void generateArrayIndices(lower::AbstractConverter &converter,
fir::FirOpBuilder &firOpBuilder,
lower::StatementContext &stmtCtx,
mlir::Location clauseLocation,
llvm::SmallVectorImpl<mlir::Value> &indices,
omp::Object object) {
auto maybeRef = evaluate::ExtractDataRef(*object.ref());
if (!maybeRef)
return;
auto *arr = std::get_if<evaluate::ArrayRef>(&maybeRef->u);
if (!arr)
return;
for (auto v : arr->subscript()) {
if (std::holds_alternative<Triplet>(v.u))
TODO(clauseLocation, "Triplet indexing in map clause is unsupported");
auto expr = std::get<Fortran::evaluate::IndirectSubscriptIntegerExpr>(v.u);
mlir::Value subscript =
fir::getBase(converter.genExprValue(toEvExpr(expr.value()), stmtCtx));
mlir::Value one = firOpBuilder.createIntegerConstant(
clauseLocation, firOpBuilder.getIndexType(), 1);
subscript = firOpBuilder.createConvert(
clauseLocation, firOpBuilder.getIndexType(), subscript);
indices.push_back(firOpBuilder.create<mlir::arith::SubIOp>(clauseLocation,
subscript, one));
}
}
/// When mapping members of derived types, there is a chance that one of the
/// members along the way to a mapped member is an descriptor. In which case
/// we have to make sure we generate a map for those along the way otherwise
/// we will be missing a chunk of data required to actually map the member
/// type to device. This function effectively generates these maps and the
/// appropriate data accesses required to generate these maps. It will avoid
/// creating duplicate maps, as duplicates are just as bad as unmapped
/// descriptor data in a lot of cases for the runtime (and unnecessary
/// data movement should be avoided where possible).
///
/// As an example for the following mapping:
///
/// type :: vertexes
/// integer(4), allocatable :: vertexx(:)
/// integer(4), allocatable :: vertexy(:)
/// end type vertexes
///
/// type :: dtype
/// real(4) :: i
/// type(vertexes), allocatable :: vertexes(:)
/// end type dtype
///
/// type(dtype), allocatable :: alloca_dtype
///
/// !$omp target map(tofrom: alloca_dtype%vertexes(N1)%vertexx)
///
/// The below HLFIR/FIR is generated (trimmed for conciseness):
///
/// On the first iteration we index into the record type alloca_dtype
/// to access "vertexes", we then generate a map for this descriptor
/// alongside bounds to indicate we only need the 1 member, rather than
/// the whole array block in this case (In theory we could map its
/// entirety at the cost of data transfer bandwidth).
///
/// %13:2 = hlfir.declare ... "alloca_dtype" ...
/// %39 = fir.load %13#0 : ...
/// %40 = fir.coordinate_of %39, %c1 : ...
/// %51 = omp.map.info var_ptr(%40 : ...) map_clauses(to) capture(ByRef) ...
/// %52 = fir.load %40 : ...
///
/// Second iteration generating access to "vertexes(N1) utilising the N1 index
/// %53 = load N1 ...
/// %54 = fir.convert %53 : (i32) -> i64
/// %55 = fir.convert %54 : (i64) -> index
/// %56 = arith.subi %55, %c1 : index
/// %57 = fir.coordinate_of %52, %56 : ...
///
/// Still in the second iteration we access the allocatable member "vertexx",
/// we return %58 from the function and provide it to the final and "main"
/// map of processMap (generated by the record type segment of the below
/// function), if this were not the final symbol in the list, i.e. we accessed
/// a member below vertexx, we would have generated the map below as we did in
/// the first iteration and then continue to generate further coordinates to
/// access further components as required.
///
/// %58 = fir.coordinate_of %57, %c0 : ...
/// %61 = omp.map.info var_ptr(%58 : ...) map_clauses(to) capture(ByRef) ...
///
/// Parent mapping containing prior generated mapped members, generated at
/// a later step but here to showcase the "end" result
///
/// omp.map.info var_ptr(%13#1 : ...) map_clauses(to) capture(ByRef)
/// members(%50, %61 : [0, 1, 0], [0, 1, 0] : ...
///
/// \param objectList - The list of omp::Object symbol data for each parent
/// to the mapped member (also includes the mapped member), generated via
/// gatherObjectsOf.
/// \param indices - List of index data associated with the mapped member
/// symbol, which identifies the placement of the member in its parent,
/// this helps generate the appropriate member accesses. These indices
/// can be generated via generateMemberPlacementIndices.
/// \param asFortran - A string generated from the mapped variable to be
/// associated with the main map, generally (but not restricted to)
/// generated via gatherDataOperandAddrAndBounds or other
/// DirectiveCommons.hpp utilities.
/// \param mapTypeBits - The map flags that will be associated with the
/// generated maps, minus alterations of the TO and FROM bits for the
/// intermediate components to prevent accidental overwriting on device
/// write back.
mlir::Value createParentSymAndGenIntermediateMaps(
mlir::Location clauseLocation, lower::AbstractConverter &converter,
semantics::SemanticsContext &semaCtx, lower::StatementContext &stmtCtx,
omp::ObjectList &objectList, llvm::SmallVectorImpl<int64_t> &indices,
OmpMapParentAndMemberData &parentMemberIndices, llvm::StringRef asFortran,
llvm::omp::OpenMPOffloadMappingFlags mapTypeBits) {
fir::FirOpBuilder &firOpBuilder = converter.getFirOpBuilder();
/// Checks if an omp::Object is an array expression with a subscript, e.g.
/// array(1,2).
auto isArrayExprWithSubscript = [](omp::Object obj) {
if (auto maybeRef = evaluate::ExtractDataRef(*obj.ref())) {
evaluate::DataRef ref = *maybeRef;
if (auto *arr = std::get_if<evaluate::ArrayRef>(&ref.u))
return !arr->subscript().empty();
}
return false;
};
// Generate the access to the original parent base address.
lower::AddrAndBoundsInfo parentBaseAddr = lower::getDataOperandBaseAddr(
converter, firOpBuilder, *objectList[0].sym(), clauseLocation);
mlir::Value curValue = parentBaseAddr.addr;
// Iterate over all objects in the objectList, this should consist of all
// record types between the parent and the member being mapped (including
// the parent). The object list may also contain array objects as well,
// this can occur when specifying bounds or a specific element access
// within a member map, we skip these.
size_t currentIndicesIdx = 0;
for (size_t i = 0; i < objectList.size(); ++i) {
// If we encounter a sequence type, i.e. an array, we must generate the
// correct coordinate operation to index into the array to proceed further,
// this is only relevant in cases where we encounter subscripts currently.
//
// For example in the following case:
//
// map(tofrom: array_dtype(4)%internal_dtypes(3)%float_elements(4))
//
// We must generate coordinate operation accesses for each subscript
// we encounter.
if (fir::SequenceType arrType = mlir::dyn_cast<fir::SequenceType>(
fir::unwrapPassByRefType(curValue.getType()))) {
if (isArrayExprWithSubscript(objectList[i])) {
llvm::SmallVector<mlir::Value> subscriptIndices;
generateArrayIndices(converter, firOpBuilder, stmtCtx, clauseLocation,
subscriptIndices, objectList[i]);
assert(!subscriptIndices.empty() &&
"missing expected indices for map clause");
curValue = firOpBuilder.create<fir::CoordinateOp>(
clauseLocation, firOpBuilder.getRefType(arrType.getEleTy()),
curValue, subscriptIndices);
}
}
// If we encounter a record type, we must access the subsequent member
// by indexing into it and creating a coordinate operation to do so, we
// utilise the index information generated previously and passed in to
// work out the correct member to access and the corresponding member
// type.
if (fir::RecordType recordType = mlir::dyn_cast<fir::RecordType>(
fir::unwrapPassByRefType(curValue.getType()))) {
mlir::Value idxConst = firOpBuilder.createIntegerConstant(
clauseLocation, firOpBuilder.getIndexType(),
indices[currentIndicesIdx]);
mlir::Type memberTy =
recordType.getTypeList().at(indices[currentIndicesIdx]).second;
curValue = firOpBuilder.create<fir::CoordinateOp>(
clauseLocation, firOpBuilder.getRefType(memberTy), curValue,
idxConst);
// Skip mapping and the subsequent load if we're the final member or not
// a type with a descriptor such as a pointer/allocatable. If we're a
// final member, the map will be generated by the processMap call that
// invoked this function, and if we're not a type with a descriptor then
// we have no need of generating an intermediate map for it, as we only
// need to generate a map if a member is a descriptor type (and thus
// obscures the members it contains via a pointer in which it's data needs
// mapped)
if ((currentIndicesIdx == indices.size() - 1) ||
!fir::isTypeWithDescriptor(memberTy)) {
currentIndicesIdx++;
continue;
}
llvm::SmallVector<int64_t> interimIndices(
indices.begin(), std::next(indices.begin(), currentIndicesIdx + 1));
// Verify we haven't already created a map for this particular member, by
// checking the list of members already mapped for the current parent,
// stored in the parentMemberIndices structure
if (!parentMemberIndices.isDuplicateMemberMapInfo(interimIndices)) {
// Generate bounds operations using the standard lowering utility,
// unfortunately this currently does a bit more than just generate
// bounds and we discard the other bits. May be useful to extend the
// utility to just provide bounds in the future.
llvm::SmallVector<mlir::Value> interimBounds;
if (i + 1 < objectList.size() &&
objectList[i + 1].sym()->IsObjectArray()) {
std::stringstream interimFortran;
Fortran::lower::gatherDataOperandAddrAndBounds<
mlir::omp::MapBoundsOp, mlir::omp::MapBoundsType>(
converter, converter.getFirOpBuilder(), semaCtx,
converter.getFctCtx(), *objectList[i + 1].sym(),
objectList[i + 1].ref(), clauseLocation, interimFortran,
interimBounds, treatIndexAsSection);
}
// Remove all map TO, FROM and TOFROM bits, from the intermediate
// allocatable maps, we simply wish to alloc or release them. It may be
// safer to just pass OMP_MAP_NONE as the map type, but we may still
// need some of the other map types the mapped member utilises, so for
// now it's good to keep an eye on this.
llvm::omp::OpenMPOffloadMappingFlags interimMapType = mapTypeBits;
interimMapType &= ~llvm::omp::OpenMPOffloadMappingFlags::OMP_MAP_TO;
interimMapType &= ~llvm::omp::OpenMPOffloadMappingFlags::OMP_MAP_FROM;
// Create a map for the intermediate member and insert it and it's
// indices into the parentMemberIndices list to track it.
mlir::omp::MapInfoOp mapOp = createMapInfoOp(
firOpBuilder, clauseLocation, curValue,
/*varPtrPtr=*/mlir::Value{}, asFortran,
/*bounds=*/interimBounds,
/*members=*/{},
/*membersIndex=*/mlir::ArrayAttr{},
static_cast<
std::underlying_type_t<llvm::omp::OpenMPOffloadMappingFlags>>(
interimMapType),
mlir::omp::VariableCaptureKind::ByRef, curValue.getType());
parentMemberIndices.memberPlacementIndices.push_back(interimIndices);
parentMemberIndices.memberMap.push_back(mapOp);
}
// Load the currently accessed member, so we can continue to access
// further segments.
curValue = firOpBuilder.create<fir::LoadOp>(clauseLocation, curValue);
currentIndicesIdx++;
}
}
return curValue;
}
static int64_t
getComponentPlacementInParent(const semantics::Symbol *componentSym) {
const auto *derived = componentSym->owner()
.derivedTypeSpec()
->typeSymbol()
.detailsIf<semantics::DerivedTypeDetails>();
assert(derived &&
"expected derived type details when processing component symbol");
for (auto [placement, name] : llvm::enumerate(derived->componentNames()))
if (name == componentSym->name())
return placement;
return -1;
}
static std::optional<Object>
getComponentObject(std::optional<Object> object,
semantics::SemanticsContext &semaCtx) {
if (!object)
return std::nullopt;
auto ref = evaluate::ExtractDataRef(*object.value().ref());
if (!ref)
return std::nullopt;
if (std::holds_alternative<evaluate::Component>(ref->u))
return object;
auto baseObj = getBaseObject(object.value(), semaCtx);
if (!baseObj)
return std::nullopt;
return getComponentObject(baseObj.value(), semaCtx);
}
void generateMemberPlacementIndices(const Object &object,
llvm::SmallVectorImpl<int64_t> &indices,
semantics::SemanticsContext &semaCtx) {
assert(indices.empty() && "indices vector passed to "
"generateMemberPlacementIndices should be empty");
auto compObj = getComponentObject(object, semaCtx);
while (compObj) {
int64_t index = getComponentPlacementInParent(compObj->sym());
assert(
index >= 0 &&
"unexpected index value returned from getComponentPlacementInParent");
indices.push_back(index);
compObj =
getComponentObject(getBaseObject(compObj.value(), semaCtx), semaCtx);
}
indices = llvm::SmallVector<int64_t>{llvm::reverse(indices)};
}
void OmpMapParentAndMemberData::addChildIndexAndMapToParent(
const omp::Object &object, mlir::omp::MapInfoOp &mapOp,
semantics::SemanticsContext &semaCtx) {
llvm::SmallVector<int64_t> indices;
generateMemberPlacementIndices(object, indices, semaCtx);
memberPlacementIndices.push_back(indices);
memberMap.push_back(mapOp);
}
bool isMemberOrParentAllocatableOrPointer(
const Object &object, semantics::SemanticsContext &semaCtx) {
if (semantics::IsAllocatableOrObjectPointer(object.sym()))
return true;
auto compObj = getBaseObject(object, semaCtx);
while (compObj) {
if (semantics::IsAllocatableOrObjectPointer(compObj.value().sym()))
return true;
compObj = getBaseObject(compObj.value(), semaCtx);
}
return false;
}
void insertChildMapInfoIntoParent(
lower::AbstractConverter &converter, semantics::SemanticsContext &semaCtx,
lower::StatementContext &stmtCtx,
std::map<Object, OmpMapParentAndMemberData> &parentMemberIndices,
llvm::SmallVectorImpl<mlir::Value> &mapOperands,
llvm::SmallVectorImpl<const semantics::Symbol *> &mapSyms) {
fir::FirOpBuilder &firOpBuilder = converter.getFirOpBuilder();
for (auto indices : parentMemberIndices) {
auto *parentIter =
llvm::find_if(mapSyms, [&indices](const semantics::Symbol *v) {
return v == indices.first.sym();
});
if (parentIter != mapSyms.end()) {
auto mapOp = llvm::cast<mlir::omp::MapInfoOp>(
mapOperands[std::distance(mapSyms.begin(), parentIter)]
.getDefiningOp());
// NOTE: To maintain appropriate SSA ordering, we move the parent map
// which will now have references to its children after the last
// of its members to be generated. This is necessary when a user
// has defined a series of parent and children maps where the parent
// precedes the children. An alternative, may be to do
// delayed generation of map info operations from the clauses and
// organize them first before generation. Or to use the
// topologicalSort utility which will enforce a stronger SSA
// dominance ordering at the cost of efficiency/time.
mapOp->moveAfter(indices.second.memberMap.back());
for (mlir::omp::MapInfoOp memberMap : indices.second.memberMap)
mapOp.getMembersMutable().append(memberMap.getResult());
mapOp.setMembersIndexAttr(firOpBuilder.create2DI64ArrayAttr(
indices.second.memberPlacementIndices));
} else {
// NOTE: We take the map type of the first child, this may not
// be the correct thing to do, however, we shall see. For the moment
// it allows this to work with enter and exit without causing MLIR
// verification issues. The more appropriate thing may be to take
// the "main" map type clause from the directive being used.
uint64_t mapType = indices.second.memberMap[0].getMapType().value_or(0);
llvm::SmallVector<mlir::Value> members;
members.reserve(indices.second.memberMap.size());
for (mlir::omp::MapInfoOp memberMap : indices.second.memberMap)
members.push_back(memberMap.getResult());
// Create parent to emplace and bind members
llvm::SmallVector<mlir::Value> bounds;
std::stringstream asFortran;
lower::AddrAndBoundsInfo info =
lower::gatherDataOperandAddrAndBounds<mlir::omp::MapBoundsOp,
mlir::omp::MapBoundsType>(
converter, firOpBuilder, semaCtx, converter.getFctCtx(),
*indices.first.sym(), indices.first.ref(),
converter.getCurrentLocation(), asFortran, bounds,
treatIndexAsSection);
mlir::omp::MapInfoOp mapOp = createMapInfoOp(
firOpBuilder, info.rawInput.getLoc(), info.rawInput,
/*varPtrPtr=*/mlir::Value(), asFortran.str(), bounds, members,
firOpBuilder.create2DI64ArrayAttr(
indices.second.memberPlacementIndices),
mapType, mlir::omp::VariableCaptureKind::ByRef,
info.rawInput.getType(),
/*partialMap=*/true);
mapOperands.push_back(mapOp);
mapSyms.push_back(indices.first.sym());
}
}
}
void lastprivateModifierNotSupported(const omp::clause::Lastprivate &lastp,
mlir::Location loc) {
using Lastprivate = omp::clause::Lastprivate;
auto &maybeMod =
std::get<std::optional<Lastprivate::LastprivateModifier>>(lastp.t);
if (maybeMod) {
assert(*maybeMod == Lastprivate::LastprivateModifier::Conditional &&
"Unexpected lastprivate modifier");
TODO(loc, "lastprivate clause with CONDITIONAL modifier");
}
}
} // namespace omp
} // namespace lower
} // namespace Fortran