[ESI] Separate out the Cap'nProto code into separate library (#382)

* Scaffolding for Capnp-dependent ESI code

* Adding 'capnp' feature

* Replicated functionality

* Just missing the complex part: schema parsing

* Parse the generated schema, get the size out of that

* Documentation

* Adding NOLINT
This commit is contained in:
John Demme 2020-12-30 23:20:03 -08:00 committed by GitHub
parent 53db980af5
commit 565bfa2b06
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 480 additions and 212 deletions

View File

@ -210,6 +210,11 @@ else()
"${CMAKE_CURRENT_SOURCE_DIR}/ext/lib/pkgconfig:$ENV{PKG_CONFIG_PATH}")
find_package(CapnProto CONFIG PATHS "${CMAKE_SOURCE_DIR}/ext")
endif()
if(CapnProto_FOUND)
set(CMAKE_INSTALL_RPATH ${capnp_LIBDIR})
set(CMAKE_INSTALL_RPATH_USE_LINK_PATH TRUE)
endif()
endif()
#-------------------------------------------------------------------------------

View File

@ -90,11 +90,6 @@ def CosimEndpoint : ESI_Physical_Op<"cosim", []> {
$clk `,` $rstn `,` $send `,` $endpointID attr-dict
`:` type($send) `->` type($recv)
}];
let extraClassDeclaration = [{
uint64_t getSendTypeID() { return getCapnpTypeID(send().getType()); }
uint64_t getRecvTypeID() { return getCapnpTypeID(recv().getType()); }
}];
}
def RtlBitArray : Type<CPred<"$_self.isa<::circt::rtl::ArrayType>()"

View File

@ -20,19 +20,6 @@
#include "ESIDialect.h"
namespace circt {
namespace esi {
/// Every time we implement a breaking change in the schema generation,
/// increment this number. It is a seed for all the schema hashes.
constexpr uint64_t esiCosimSchemaVersion = 1;
/// Get the Cap'nProto schema ID for a type.
uint64_t getCapnpTypeID(mlir::Type);
} // namespace esi
} // namespace circt
#define GET_TYPEDEF_CLASSES
#include "circt/Dialect/ESI/ESITypes.h.inc"

View File

@ -105,4 +105,8 @@ if config.esi_cosim_path != "":
config.substitutions.append(('%ESIINC%', f'{config.circt_include_dir}/circt/Dialect/ESI/'))
config.substitutions.append(('%ESICOSIM%', f'{config.esi_cosim_path}'))
# Enable ESI's Capnp tests if they're supported.
if config.esi_capnp != "":
config.available_features.add('capnp')
llvm_config.add_tool_substitutions(tools, tool_dirs)

View File

@ -41,6 +41,7 @@ config.esi_cosim_path = "@ESI_COSIM_PATH@"
config.timeout = "@CIRCT_INTEGRATION_TIMEOUT@"
config.yosys_path = "@YOSYS_PATH@"
config.questa_path = "@QUESTA_PATH@"
config.esi_capnp = "@ESI_CAPNP@"
# Support substitution of the tools_dir with user parameters. This is
# used when we can't determine the tool dir at configuration time.

View File

@ -1,3 +1,23 @@
add_subdirectory(capnp)
set(ESI_Deps
CIRCTSupport
MLIRESIEnumsIncGen
)
set(ESI_LinkLibs
CIRCTSupport
CIRCTSV
MLIREDSC
MLIRIR
MLIRTransforms
)
if (ESI_CAPNP)
list(APPEND ESI_Deps ESICapnp)
list(APPEND ESI_LinkLibs ESICapnp)
endif()
add_circt_dialect_library(CIRCTESI
ESIDialect.cpp
ESIOps.cpp
@ -9,23 +29,18 @@ add_circt_dialect_library(CIRCTESI
${CIRCT_MAIN_INCLUDE_DIR}/circt/Dialect/ESI
DEPENDS
CIRCTSupport
MLIRESIIncGen
MLIRESIEnumsIncGen
${ESI_Deps}
LINK_COMPONENTS
Core
Support
LINK_LIBS PUBLIC
CIRCTSupport
CIRCTSV
MLIREDSC
MLIRIR
MLIRTransforms
${ESI_LinkLibs}
)
add_dependencies(circt-headers MLIRESIIncGen)
if (ESI_CAPNP)
target_compile_definitions(obj.CIRCTESI PRIVATE CAPNP)
endif()
add_subdirectory(cosim)

View File

@ -26,6 +26,10 @@
#include <memory>
#ifdef CAPNP
#include "capnp/ESICapnp.h"
#endif
namespace circt {
namespace esi {
#define GEN_PASS_CLASSES
@ -709,43 +713,6 @@ LogicalResult UnwrapInterfaceLower::matchAndRewrite(
return success();
}
// Compute the expected size of the capnp message field in bits. Return -1 on
// non-representable type.
static ssize_t getCapnpMsgFieldSize(Type type) {
return llvm::TypeSwitch<::mlir::Type, int64_t>(type)
.Case<IntegerType>([](IntegerType t) {
auto w = t.getWidth();
if (w == 1)
return 8;
else if (w <= 8)
return 8;
else if (w <= 16)
return 16;
else if (w <= 32)
return 32;
else if (w <= 64)
return 64;
return -1;
})
.Default([](Type) { return -1; });
}
// Compute the expected size of the capnp message in bits. Return -1 on
// non-representable type. TODO: replace this with a call into the Capnp C++
// library to parse a schema and have it compute sizes and offsets.
static ssize_t getCapnpMsgSize(Type type) {
ChannelPort chan = type.dyn_cast<ChannelPort>();
if (chan)
return getCapnpMsgSize(chan.getInner());
ssize_t headerSize = 128;
ssize_t fieldSize = getCapnpMsgFieldSize(type);
if (fieldSize < 0)
return fieldSize;
// Capnp sizes are always multiples of 8 bytes, so round up.
fieldSize = (fieldSize & ~0x3f) + (fieldSize & 0x3f ? 0x40 : 0);
return headerSize + fieldSize;
}
namespace {
/// Lower `CosimEndpoint` ops to a SystemVerilog extern module and a Capnp
/// gasket op.
@ -768,26 +735,34 @@ private:
LogicalResult
CosimLowering::matchAndRewrite(CosimEndpoint ep, ArrayRef<Value> operands,
ConversionPatternRewriter &rewriter) const {
#ifndef CAPNP
return rewriter.notifyMatchFailure(
ep, "Cosim lowering requires the ESI capnp plugin, which was disabled.");
#else
auto loc = ep.getLoc();
auto *ctxt = rewriter.getContext();
circt::BackedgeBuilder bb(rewriter, loc);
builder.declareCosimEndpoint();
Type ui64Type =
IntegerType::get(ctxt, 64, IntegerType::SignednessSemantics::Unsigned);
capnp::TypeSchema sendTypeSchema(ep.send().getType());
capnp::TypeSchema recvTypeSchema(ep.recv().getType());
// Set all the parameters.
NamedAttrList params;
params.set("ENDPOINT_ID", rewriter.getI32IntegerAttr(ep.endpointID()));
params.set("SEND_TYPE_ID", IntegerAttr::get(ui64Type, ep.getSendTypeID()));
params.set("SEND_TYPE_ID",
IntegerAttr::get(ui64Type, sendTypeSchema.capnpTypeID()));
params.set("SEND_TYPE_SIZE_BITS",
rewriter.getI64IntegerAttr(getCapnpMsgSize(ep.send().getType())));
params.set("RECV_TYPE_ID", IntegerAttr::get(ui64Type, ep.getRecvTypeID()));
rewriter.getI64IntegerAttr(sendTypeSchema.size()));
params.set("RECV_TYPE_ID",
IntegerAttr::get(ui64Type, recvTypeSchema.capnpTypeID()));
params.set("RECVTYPE_SIZE_BITS",
rewriter.getI64IntegerAttr(getCapnpMsgSize(ep.recv().getType())));
rewriter.getI64IntegerAttr(recvTypeSchema.size()));
// Set up the egest route to drive the EP's send ports.
ArrayType egestBitArrayType = ArrayType::get(
rewriter.getI1Type(), getCapnpMsgSize(ep.send().getType()));
ArrayType egestBitArrayType =
ArrayType::get(rewriter.getI1Type(), sendTypeSchema.size());
auto sendReady = bb.get(rewriter.getI1Type());
UnwrapValidReady unwrapSend =
rewriter.create<UnwrapValidReady>(loc, ep.send(), sendReady);
@ -796,8 +771,8 @@ CosimLowering::matchAndRewrite(CosimEndpoint ep, ArrayRef<Value> operands,
// Get information necessary for injest path.
auto recvReady = bb.get(rewriter.getI1Type());
ArrayType ingestBitArrayType = ArrayType::get(
rewriter.getI1Type(), getCapnpMsgSize(ep.recv().getType()));
ArrayType ingestBitArrayType =
ArrayType::get(rewriter.getI1Type(), recvTypeSchema.size());
// Create replacement Cosim_Endpoint instance.
StringAttr nameAttr = ep.getAttr("name").dyn_cast_or_null<StringAttr>();
@ -829,6 +804,7 @@ CosimLowering::matchAndRewrite(CosimEndpoint ep, ArrayRef<Value> operands,
rewriter.replaceOp(ep, wrapRecv.chanOutput());
return success();
#endif // CAPNP
}
void ESItoRTLPass::runOnOperation() {

View File

@ -16,10 +16,12 @@
#include "mlir/IR/BuiltinOps.h"
#include "mlir/Translation.h"
#include "llvm/ADT/TypeSwitch.h"
#include <algorithm>
#ifdef CAPNP
#include "capnp/ESICapnp.h"
#endif
using namespace mlir;
using namespace circt::esi;
@ -27,9 +29,11 @@ using namespace circt::esi;
// ESI Cosim Cap'nProto schema generation.
//
// Cosimulation in ESI is done over capnp. This translation walks the IR, finds
// all the `esi.cosim` ops, and creates a schema for all the types.
// all the `esi.cosim` ops, and creates a schema for all the types. It requires
// CAPNP to be enabled.
//===----------------------------------------------------------------------===//
#ifdef CAPNP
namespace {
struct ExportCosimSchema {
ExportCosimSchema(ModuleOp module, llvm::raw_ostream &os)
@ -54,22 +58,7 @@ struct ExportCosimSchema {
/// comments.
LogicalResult visitEndpoint(CosimEndpoint);
/// Do we support emitting a schema for 'type'?
static bool isTypeSupported(Type type);
/// Emit a schema for a single int.
LogicalResult emitSchemaFor(IntegerType type, uint64_t hash);
/// Emit a struct name.
llvm::raw_ostream &emitName(Type type) { return os << "TY" << type; }
private:
/// Intentation utils.
raw_ostream &indent() { return os.indent(currentIndent); }
void addIndent() { currentIndent += 2; }
void reduceIndent() { currentIndent -= 2; }
size_t currentIndent = 0;
ModuleOp module;
llvm::raw_ostream &os;
DiagnosticEngine &diag;
@ -78,88 +67,34 @@ private:
// All the `esi.cosim` input and output types encountered during the IR walk.
// This is NOT in a deterministic order!
llvm::SmallVector<ChannelPort> types;
llvm::SmallVector<capnp::TypeSchema> types;
};
} // anonymous namespace
bool ExportCosimSchema::isTypeSupported(Type type) {
auto chanPort = type.dyn_cast<ChannelPort>();
if (chanPort) {
return isTypeSupported(chanPort.getInner());
}
auto i = type.dyn_cast<IntegerType>();
if (i)
return i.getWidth() <= 64;
return false;
}
LogicalResult ExportCosimSchema::emitSchemaFor(IntegerType type,
uint64_t hash) {
SmallString<16> typeStr;
if (type.isSigned())
typeStr = "Int";
else
typeStr = "UInt";
// Round up.
auto w = type.getWidth();
if (w == 1)
typeStr = "Bool";
else if (w <= 8)
typeStr += "8";
else if (w <= 16)
typeStr += "16";
else if (w <= 32)
typeStr += "32";
else if (w <= 64)
typeStr += "64";
else
return diag.emit(unknown, DiagnosticSeverity::Error)
<< "Capnp does not support ints larger than 64-bit";
// Since capnp requires messages to be structs, emit a wrapper struct.
indent() << "struct ";
emitName(type) << " ";
emitId(hash) << " {\n";
addIndent();
// Specify the actual type, followed by the capnp field.
indent() << "# Actual type is " << type << ".\n";
indent() << "i @0 :" << typeStr << ";\n";
reduceIndent();
indent() << "}\n\n";
return success();
}
LogicalResult ExportCosimSchema::visitEndpoint(CosimEndpoint ep) {
ChannelPort sendPort = ep.send().getType().dyn_cast<ChannelPort>();
if (!sendPort)
return ep.emitOpError("Expected ChannelPort type for input. Got ")
<< sendPort;
if (!isTypeSupported(sendPort))
return ep.emitOpError("Type '") << sendPort << "' not supported.";
types.push_back(sendPort);
capnp::TypeSchema sendTypeSchema(ep.send().getType());
if (!sendTypeSchema.isSupported())
return ep.emitOpError("Type ") << ep.send().getType() << " not supported.";
types.push_back(sendTypeSchema);
ChannelPort recvPort = ep.recv().getType().dyn_cast<ChannelPort>();
if (!recvPort)
ep.emitOpError("Expected ChannelPort type for output. Got ") << recvPort;
if (!isTypeSupported(recvPort))
return ep.emitOpError("Type '") << recvPort << "' not supported.";
types.push_back(recvPort);
capnp::TypeSchema recvTypeSchema(ep.recv().getType());
if (!recvTypeSchema.isSupported())
return ep.emitOpError("Type '")
<< ep.recv().getType() << "' not supported.";
types.push_back(recvTypeSchema);
os << "# Endpoint ";
StringAttr epName = ep.getAttrOfType<StringAttr>("name");
if (epName)
os << epName << " is endpoint ";
os << "#" << ep.endpointID() << " at " << ep.getLoc() << ":\n";
os << "# Input type: ";
emitName(sendPort.getInner()) << " ";
emitId(ep.getSendTypeID()) << "\n";
os << "# Output type: ";
emitName(recvPort.getInner()) << " ";
emitId(ep.getRecvTypeID()) << "\n";
os << "# Send type: ";
sendTypeSchema.writeMetadata(os);
os << "\n";
os << "# Recv type: ";
recvTypeSchema.writeMetadata(os);
os << "\n";
return success();
}
@ -174,15 +109,16 @@ LogicalResult ExportCosimSchema::emit() {
os << "#########################################################\n";
// We need a sorted list to ensure determinism.
llvm::sort(types.begin(), types.end(), [](ChannelPort a, ChannelPort b) {
return getCapnpTypeID(a) > getCapnpTypeID(b);
});
llvm::sort(types.begin(), types.end(),
[](capnp::TypeSchema &a, capnp::TypeSchema &b) {
return a.capnpTypeID() > b.capnpTypeID();
});
// Compute and emit the capnp file id.
uint64_t fileHash = 2544816649379317016; // Some random number.
for (ChannelPort chanPort : types)
fileHash = llvm::hashing::detail::hash_16_bytes(fileHash,
getCapnpTypeID(chanPort));
for (capnp::TypeSchema &schema : types)
fileHash =
llvm::hashing::detail::hash_16_bytes(fileHash, schema.capnpTypeID());
// Capnp IDs always have a '1' high bit.
fileHash |= 0x8000000000000000;
emitId(fileHash) << ";\n\n";
@ -190,19 +126,7 @@ LogicalResult ExportCosimSchema::emit() {
// Iterate through the various types and emit their schemas.
auto end = std::unique(types.begin(), types.end());
for (auto typeIter = types.begin(); typeIter < end; ++typeIter) {
ChannelPort chanPort = *typeIter;
uint64_t typeHash = getCapnpTypeID(chanPort);
LogicalResult rc =
TypeSwitch<Type, LogicalResult>(chanPort.getInner())
.Case([this, typeHash](IntegerType it) {
return emitSchemaFor(it, typeHash);
})
.Default([](Type type) {
assert(false && "Unsupported type should have been filtered out "
"in visitEndpoint().");
return failure();
});
if (failed(rc))
if (failed(typeIter->write(os)))
// If we fail during an emission, dump out early since the output may be
// corrupted.
return failure();
@ -216,15 +140,19 @@ static LogicalResult exportCosimSchema(ModuleOp module, llvm::raw_ostream &os) {
return schema.emit();
}
#endif
//===----------------------------------------------------------------------===//
// Register all ESI translations.
//===----------------------------------------------------------------------===//
void circt::esi::registerESITranslations() {
#ifdef CAPNP
TranslateFromMLIRRegistration cosimToCapnp(
"emit-esi-capnp", exportCosimSchema, [](DialectRegistry &registry) {
registry
.insert<ESIDialect, circt::rtl::RTLDialect, circt::sv::SVDialect,
mlir::StandardOpsDialect, mlir::BuiltinDialect>();
});
#endif
}

View File

@ -15,42 +15,11 @@
#include "mlir/IR/Attributes.h"
#include "mlir/IR/DialectImplementation.h"
#include "llvm/ADT/ArrayRef.h"
#include "llvm/ADT/Hashing.h"
#include "llvm/ADT/TypeSwitch.h"
using namespace mlir;
using namespace circt::esi;
// We compute a deterministic hash based on the type. Since llvm::hash_value
// changes from execution to execution, we don't use it. This assumes a closed
// type system, which is reasonable since we only support some types in the
// Capnp schema generation anyway.
uint64_t circt::esi::getCapnpTypeID(Type t) {
// We can hash up to 64 bytes with a single function call.
char buffer[64];
memset(buffer, 0, sizeof(buffer));
// The first byte is for the outer type.
auto chanPort = t.dyn_cast<ChannelPort>();
assert(chanPort && "Type not supported as top level");
buffer[0] = 1; // Constant for the ChannelPort type.
TypeSwitch<Type>(chanPort.getInner())
.Case([&buffer](IntegerType t) {
// The second byte is for the inner type.
buffer[1] = 1;
// The rest can be defined arbitrarily.
buffer[2] = (char)t.getSignedness();
*(int64_t *)&buffer[4] = t.getWidth();
})
.Default([](Type) { assert(false && "Type not yet supported"); });
uint64_t hash =
llvm::hashing::detail::hash_short(buffer, 12, esiCosimSchemaVersion);
// Capnp IDs always have a '1' high bit.
return hash | 0x8000000000000000;
}
Type ChannelPort::parse(mlir::MLIRContext *ctxt, mlir::DialectAsmParser &p) {
Type inner;
if (p.parseLess() || p.parseType(inner) || p.parseGreater())

View File

@ -0,0 +1,19 @@
##===- CMakeLists.txt - ESI code which needs capnp ------------*- cmake -*-===//
##
## Parts of the ESI dialect code need access to libcapnp and libcapnpc, which
## may not be available. Put all that code in here to make it an optional
## feature.
##
##===----------------------------------------------------------------------===//
if(CapnProto_FOUND)
option(ESI_CAPNP "Enable ESI Capnp features" ON)
add_circt_library(ESICapnp
Schema.cpp
LINK_LIBS PUBLIC
CapnProto::capnp CapnProto::capnpc
)
target_link_libraries(obj.ESICapnp CapnProto::capnp CapnProto::capnpc)
endif()

View File

@ -0,0 +1,74 @@
//===- ESICapnp.h - ESI Cap'nProto library utilies --------------*- 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
//
//===----------------------------------------------------------------------===//
//
// ESI utility code which requires libcapnp and libcapnpc.
//
//===----------------------------------------------------------------------===//
// NOLINTNEXTLINE(llvm-header-guard)
#ifndef CIRCT_DIALECT_ESI_CAPNP_ESICAPNP_H
#define CIRCT_DIALECT_ESI_CAPNP_ESICAPNP_H
#include <memory>
namespace mlir {
class Type;
struct LogicalResult;
} // namespace mlir
namespace llvm {
class raw_ostream;
class StringRef;
} // namespace llvm
namespace circt {
namespace esi {
namespace capnp {
/// Every time we implement a breaking change in the schema generation,
/// increment this number. It is a seed for all the schema hashes.
constexpr uint64_t esiCosimSchemaVersion = 1;
namespace detail {
struct TypeSchemaImpl;
} // namespace detail
/// Generate and reason about a Cap'nProto schema for a particular MLIR type.
class TypeSchema {
public:
TypeSchema(mlir::Type);
bool operator==(const TypeSchema &) const;
/// Get the Cap'nProto schema ID for a type.
uint64_t capnpTypeID() const;
/// Returns true if the type is currently supported.
bool isSupported() const;
/// Size in bits of the capnp message.
size_t size() const;
/// Get the capnp struct name.
llvm::StringRef name() const;
/// Write out the name and ID in capnp schema format.
mlir::LogicalResult writeMetadata(llvm::raw_ostream &os) const;
/// Write out the schema in its entirety.
mlir::LogicalResult write(llvm::raw_ostream &os) const;
private:
/// The implementation of this. Separate to hide the details and avoid having
/// to include the capnp headers in this header.
std::shared_ptr<detail::TypeSchemaImpl> s;
};
} // namespace capnp
} // namespace esi
} // namespace circt
#endif // CIRCT_DIALECT_ESI_CAPNP_ESICAPNP_H

View File

@ -0,0 +1,289 @@
//===- Schema.cpp - ESI Cap'nProto schema utilities -------------*- 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
//
//===----------------------------------------------------------------------===//
//
//
//===----------------------------------------------------------------------===//
#include "ESICapnp.h"
#include "circt/Dialect/ESI/ESITypes.h"
#include "capnp/schema-parser.h"
#include "mlir/IR/BuiltinTypes.h"
#include "llvm/ADT/TypeSwitch.h"
#include "llvm/Support/Format.h"
#include <string>
using namespace mlir;
using namespace circt::esi::capnp::detail;
//===----------------------------------------------------------------------===//
// Utilities.
//===----------------------------------------------------------------------===//
namespace {
/// Intentation utils.
class IndentingOStream {
public:
IndentingOStream(llvm::raw_ostream &os) : os(os) {}
template <typename T>
IndentingOStream &operator<<(T t) {
os << t;
return *this;
}
IndentingOStream &indent() {
os.indent(currentIndent);
return *this;
}
void addIndent() { currentIndent += 2; }
void reduceIndent() { currentIndent -= 2; }
private:
llvm::raw_ostream &os;
size_t currentIndent = 0;
};
} // namespace
/// Emit an ID in capnp format.
static llvm::raw_ostream &emitId(llvm::raw_ostream &os, int64_t id) {
return os << "@" << llvm::format_hex(id, /*width=*/16 + 2);
}
//===----------------------------------------------------------------------===//
// TypeSchema class implementation.
//===----------------------------------------------------------------------===//
namespace circt {
namespace esi {
namespace capnp {
namespace detail {
/// Actual implementation of `TypeSchema` to keep all the details out of the
/// header.
struct TypeSchemaImpl {
public:
TypeSchemaImpl(Type type) : type(type) {}
/// Get the Cap'nProto schema ID for a type.
uint64_t capnpTypeID() const;
bool isSupported() const;
size_t size() const;
StringRef name() const;
mlir::LogicalResult write(llvm::raw_ostream &os) const;
mlir::LogicalResult writeMetadata(llvm::raw_ostream &os) const;
bool operator==(const TypeSchemaImpl &) const;
private:
::capnp::ParsedSchema getSchema() const;
::capnp::StructSchema getTypeSchema() const;
Type type;
::capnp::SchemaParser parser;
mutable llvm::Optional<uint64_t> cachedID;
mutable std::string cachedName;
mutable ::capnp::ParsedSchema rootSchema;
mutable ::capnp::StructSchema typeSchema;
};
} // namespace detail
} // namespace capnp
} // namespace esi
} // namespace circt
/// Write a valid capnp schema to memory, then parse it out of memory using the
/// capnp library. Writing and parsing text within a single process is ugly, but
/// this is by far the easiest way to do this. This isn't the use case for which
/// Cap'nProto was designed.
::capnp::ParsedSchema TypeSchemaImpl::getSchema() const {
if (rootSchema != ::capnp::ParsedSchema())
return rootSchema;
// Write the schema to `schemaText`.
std::string schemaText;
llvm::raw_string_ostream os(schemaText);
emitId(os, 0xFFFFFFFFFFFFFFFF) << ";\n";
auto rc = write(os);
assert(succeeded(rc) && "Failed schema text output.");
os.str();
// Write `schemaText` to an in-memory filesystem then parse it. Yes, this is
// the only way to do this.
kj::Own<kj::Filesystem> fs = kj::newDiskFilesystem();
kj::Own<kj::Directory> dir = kj::newInMemoryDirectory(kj::nullClock());
kj::Path fakePath = kj::Path::parse("schema.capnp");
{ // Ensure that 'fakeFile' has flushed.
auto fakeFile = dir->openFile(fakePath, kj::WriteMode::CREATE);
fakeFile->writeAll(schemaText);
}
rootSchema = parser.parseFromDirectory(*dir, std::move(fakePath), nullptr);
return rootSchema;
}
/// Find the schema corresponding to `type` and return it.
::capnp::StructSchema TypeSchemaImpl::getTypeSchema() const {
if (typeSchema != ::capnp::StructSchema())
return typeSchema;
uint64_t id = capnpTypeID();
for (auto schemaNode : getSchema().getAllNested()) {
if (schemaNode.getProto().getId() == id) {
typeSchema = schemaNode.asStruct();
return typeSchema;
}
}
assert(false && "A node with a matching ID should always be found.");
}
// We compute a deterministic hash based on the type. Since llvm::hash_value
// changes from execution to execution, we don't use it. This assumes a closed
// type system, which is reasonable since we only support some types in the
// Capnp schema generation anyway.
uint64_t TypeSchemaImpl::capnpTypeID() const {
if (cachedID)
return *cachedID;
// We can hash up to 64 bytes with a single function call.
char buffer[64];
memset(buffer, 0, sizeof(buffer));
// The first byte is for the outer type.
buffer[0] = 1; // Constant for the ChannelPort type.
TypeSwitch<Type>(type)
.Case([&buffer](IntegerType t) {
// The second byte is for the inner type.
buffer[1] = 1;
// The rest can be defined arbitrarily.
buffer[2] = (char)t.getSignedness();
*(int64_t *)&buffer[4] = t.getWidth();
})
.Default([](Type) { assert(false && "Type not yet supported"); });
uint64_t hash =
llvm::hashing::detail::hash_short(buffer, 12, esiCosimSchemaVersion);
// Capnp IDs always have a '1' high bit.
cachedID = hash | 0x8000000000000000;
return *cachedID;
}
/// Returns true if the type is currently supported.
bool TypeSchemaImpl::isSupported() const {
return llvm::TypeSwitch<::mlir::Type, bool>(type)
.Case<IntegerType>([](IntegerType t) { return t.getWidth() <= 64; })
.Default([](Type) { return false; });
}
// Compute the expected size of the capnp message in bits.
size_t TypeSchemaImpl::size() const {
auto schema = getTypeSchema();
auto structProto = schema.getProto().getStruct();
return 64 * // Convert from 64-bit words to bits.
(2 + // Headers
structProto.getDataWordCount() + structProto.getPointerCount());
}
/// For now, the name is just the type serialized. This works only because we
/// only support ints.
StringRef TypeSchemaImpl::name() const {
if (cachedName == "") {
llvm::raw_string_ostream os(cachedName);
os << "TY" << type;
cachedName = os.str();
}
return cachedName;
}
/// This function is essentially a placeholder which only supports ints. It'll
/// need to be re-worked when we start supporting structs, arrays, unions,
/// enums, etc.
mlir::LogicalResult TypeSchemaImpl::write(llvm::raw_ostream &rawOS) const {
IndentingOStream os(rawOS);
// Since capnp requires messages to be structs, emit a wrapper struct.
os.indent() << "struct ";
writeMetadata(rawOS);
os << " {\n";
os.addIndent();
auto intTy = type.dyn_cast<IntegerType>();
assert(intTy &&
"Type not supported. Please check support first with isSupported()");
// Specify the actual type, followed by the capnp field.
os.indent() << "# Actual type is " << type << ".\n";
os.indent() << "i @0 :";
auto w = intTy.getWidth();
if (w == 1) {
os.indent() << "Bool";
} else {
if (intTy.isSigned())
os << "Int";
else
os << "UInt";
// Round up.
if (w <= 8)
os << "8";
else if (w <= 16)
os << "16";
else if (w <= 32)
os << "32";
else if (w <= 64)
os << "64";
else
assert(false && "Type not supported. Please check support first with "
"isSupported()");
}
os << ";\n";
os.reduceIndent();
os.indent() << "}\n\n";
return success();
}
mlir::LogicalResult TypeSchemaImpl::writeMetadata(llvm::raw_ostream &os) const {
os << name() << " ";
emitId(os, capnpTypeID());
return success();
}
bool TypeSchemaImpl::operator==(const TypeSchemaImpl &that) const {
return type == that.type;
}
//===----------------------------------------------------------------------===//
// TypeSchema wrapper.
//===----------------------------------------------------------------------===//
circt::esi::capnp::TypeSchema::TypeSchema(Type type) {
circt::esi::ChannelPort chan = type.dyn_cast<circt::esi::ChannelPort>();
if (chan) // Unwrap the channel if it's a channel.
type = chan.getInner();
s = std::make_shared<detail::TypeSchemaImpl>(type);
}
uint64_t circt::esi::capnp::TypeSchema::capnpTypeID() const {
return s->capnpTypeID();
}
bool circt::esi::capnp::TypeSchema::isSupported() const {
return s->isSupported();
}
size_t circt::esi::capnp::TypeSchema::size() const { return s->size(); }
StringRef circt::esi::capnp::TypeSchema::name() const { return s->name(); }
mlir::LogicalResult
circt::esi::capnp::TypeSchema::write(llvm::raw_ostream &os) const {
return s->write(os);
}
mlir::LogicalResult
circt::esi::capnp::TypeSchema::writeMetadata(llvm::raw_ostream &os) const {
return s->writeMetadata(os);
}
bool circt::esi::capnp::TypeSchema::operator==(const TypeSchema &that) const {
return *s == *that.s;
}

View File

@ -1,3 +1,4 @@
// REQUIRES: capnp
// RUN: circt-opt %s -verify-diagnostics | circt-opt -verify-diagnostics | FileCheck %s
// RUN: circt-opt %s --lower-esi-to-rtl -verify-diagnostics | circt-opt -verify-diagnostics | FileCheck --check-prefix=COSIM %s
// RUN: circt-translate %s -emit-esi-capnp -verify-diagnostics | FileCheck --check-prefix=CAPNP %s

View File

@ -69,4 +69,8 @@ if config.verilator_path != "":
tools.append('verilator')
config.available_features.add('verilator')
# Enable ESI's Capnp tests if they're supported.
if config.esi_capnp != "":
config.available_features.add('capnp')
llvm_config.add_tool_substitutions(tools, tool_dirs)

View File

@ -35,6 +35,7 @@ config.circt_src_root = "@CIRCT_SOURCE_DIR@"
config.circt_obj_root = "@CIRCT_BINARY_DIR@"
config.circt_tools_dir = "@CIRCT_TOOLS_DIR@"
config.verilator_path = "@VERILATOR_PATH@"
config.esi_capnp = "@ESI_CAPNP@"
# Support substitution of the tools_dir with user parameters. This is
# used when we can't determine the tool dir at configuration time.