Add custom_insts for our own OpExtInsts, and use it for some debuginfo.

This commit is contained in:
Eduard-Mihai Burtescu 2023-05-26 06:40:02 +03:00 committed by Eduard-Mihai Burtescu
parent 69349b1b9d
commit c5dcd035b1
13 changed files with 618 additions and 25 deletions

View File

@ -1,6 +1,7 @@
use super::Builder;
use crate::abi::ConvSpirvType;
use crate::builder_spirv::{BuilderCursor, SpirvConst, SpirvValue, SpirvValueExt, SpirvValueKind};
use crate::custom_insts::{CustomInst, CustomOp};
use crate::rustc_codegen_ssa::traits::BaseTypeMethods;
use crate::spirv_type::SpirvType;
use rspirv::dr::{InsertPoint, Instruction, Operand};
@ -656,15 +657,48 @@ impl<'a, 'tcx> BuilderMethods<'a, 'tcx> for Builder<'a, 'tcx> {
}
fn set_span(&mut self, span: Span) {
self.current_span = Some(span);
let old_span = self.current_span.replace(span);
// We may not always have valid spans.
// FIXME(eddyb) reduce the sources of this as much as possible.
if span.is_dummy() {
self.emit().no_line();
// FIXME(eddyb) enable this once cross-block interactions are figured out
// (in particular, every block starts off with no debuginfo active).
if false {
// Avoid redundant debuginfo.
if old_span == Some(span) {
return;
}
}
// HACK(eddyb) this is only to aid testing (and to not remove the old code).
let use_custom_insts = true;
if use_custom_insts {
// FIXME(eddyb) this should be cached more efficiently.
let void_ty = SpirvType::Void.def(rustc_span::DUMMY_SP, self);
// We may not always have valid spans.
// FIXME(eddyb) reduce the sources of this as much as possible.
if span.is_dummy() {
self.custom_inst(void_ty, CustomInst::ClearDebugSrcLoc);
} else {
let (file, line, col) = self.builder.file_line_col_for_op_line(span);
self.custom_inst(
void_ty,
CustomInst::SetDebugSrcLoc {
file: Operand::IdRef(file.file_name_op_string_id),
line: Operand::IdRef(self.const_u32(line).def(self)),
col: Operand::IdRef(self.const_u32(col).def(self)),
},
);
}
} else {
let (file, line, col) = self.builder.file_line_col_for_op_line(span);
self.emit().line(file.file_name_op_string_id, line, col);
// We may not always have valid spans.
// FIXME(eddyb) reduce the sources of this as much as possible.
if span.is_dummy() {
self.emit().no_line();
} else {
let (file, line, col) = self.builder.file_line_col_for_op_line(span);
self.emit().line(file.file_name_op_string_id, line, col);
}
}
}
@ -2359,6 +2393,8 @@ impl<'a, 'tcx> BuilderMethods<'a, 'tcx> for Builder<'a, 'tcx> {
_ => return None,
};
let custom_ext_inst_set_import = self.ext_inst.borrow_mut().import_custom(self);
// HACK(eddyb) we can remove SSA instructions even when they have
// side-effects, *as long as* they are "local" enough and cannot
// be observed from outside this current invocation - because the
@ -2372,7 +2408,14 @@ impl<'a, 'tcx> BuilderMethods<'a, 'tcx> for Builder<'a, 'tcx> {
.instructions
.iter()
.enumerate()
.filter(|(_, inst)| ![Op::Line, Op::NoLine].contains(&inst.class.opcode));
.filter(|(_, inst)| {
let is_standard_debug = [Op::Line, Op::NoLine].contains(&inst.class.opcode);
let is_custom_debug = inst.class.opcode == Op::ExtInst
&& inst.operands[0].unwrap_id_ref() == custom_ext_inst_set_import
&& [CustomOp::SetDebugSrcLoc, CustomOp::ClearDebugSrcLoc]
.contains(&CustomOp::decode_from_ext_inst(inst));
!(is_standard_debug || is_custom_debug)
});
let mut relevant_insts_next_back = |expected_op| {
non_debug_insts
.next_back()

View File

@ -1,5 +1,6 @@
use super::Builder;
use crate::builder_spirv::{SpirvValue, SpirvValueExt};
use crate::custom_insts;
use rspirv::spirv::{GLOp, Word};
use rspirv::{dr::Operand, spirv::Capability};
@ -8,11 +9,26 @@ const GLSL_STD_450: &str = "GLSL.std.450";
/// Manager for OpExtInst/OpExtImport instructions
#[derive(Default)]
pub struct ExtInst {
/// See `crate::custom_insts` for more details on what this entails.
custom: Option<Word>,
glsl: Option<Word>,
integer_functions_2_intel: bool,
}
impl ExtInst {
pub fn import_custom(&mut self, bx: &Builder<'_, '_>) -> Word {
if let Some(id) = self.custom {
id
} else {
let id = bx
.emit_global()
.ext_inst_import(custom_insts::CUSTOM_EXT_INST_SET.clone());
self.custom = Some(id);
id
}
}
pub fn import_glsl(&mut self, bx: &Builder<'_, '_>) -> Word {
if let Some(id) = self.glsl {
id
@ -46,6 +62,24 @@ impl ExtInst {
}
impl<'a, 'tcx> Builder<'a, 'tcx> {
pub fn custom_inst(
&mut self,
result_type: Word,
inst: custom_insts::CustomInst<Operand>,
) -> SpirvValue {
let custom_ext_inst_set = self.ext_inst.borrow_mut().import_custom(self);
self.emit()
.ext_inst(
result_type,
None,
custom_ext_inst_set,
inst.op() as u32,
inst.into_operands(),
)
.unwrap()
.with_type(result_type)
}
pub fn gl_op(
&mut self,
op: GLOp,

View File

@ -369,6 +369,11 @@ impl CodegenArgs {
"dump the SPIR-T module across passes, to a (pair of) file(s) in DIR",
"DIR",
);
opts.optflag(
"",
"spirt-keep-custom-debuginfo-in-dumps",
"keep custom debuginfo when dumping SPIR-T (instead of lossily prettifying it)",
);
opts.optflag(
"",
"specializer-debug",
@ -534,6 +539,8 @@ impl CodegenArgs {
dump_post_merge: matches_opt_dump_dir_path("dump-post-merge"),
dump_post_split: matches_opt_dump_dir_path("dump-post-split"),
dump_spirt_passes: matches_opt_dump_dir_path("dump-spirt-passes"),
spirt_keep_custom_debuginfo_in_dumps: matches
.opt_present("spirt-keep-custom-debuginfo-in-dumps"),
specializer_debug: matches.opt_present("specializer-debug"),
specializer_dump_instances: matches_opt_path("specializer-dump-instances"),
print_all_zombie: matches.opt_present("print-all-zombie"),

View File

@ -2,6 +2,7 @@
//! the original codegen of a crate, and consumed by the `linker`.
use crate::builder_spirv::BuilderSpirv;
use crate::custom_insts::{self, CustomInst};
use either::Either;
use itertools::Itertools;
use rspirv::dr::{Instruction, Module, Operand};
@ -201,6 +202,14 @@ pub struct SpanRegenerator<'a> {
#[derive(Default)]
struct SpvDebugInfo<'a> {
/// ID of `OpExtInstImport` for our custom "extended instruction set",
/// if present (see `crate::custom_insts` for more details).
custom_ext_inst_set_import: Option<Word>,
// HACK(eddyb) this is only needed because `OpExtInst`s can't have immediates,
// and must resort to referencing `OpConstant`s instead.
id_to_op_constant_operand: FxIndexMap<Word, &'a Operand>,
id_to_op_string: FxIndexMap<Word, &'a str>,
files: FxIndexMap<&'a str, SpvDebugFile<'a>>,
}
@ -235,6 +244,24 @@ impl<'a> SpvDebugInfo<'a> {
}
};
// FIXME(eddyb) avoid repeating this across different passes/helpers.
this.custom_ext_inst_set_import = module
.ext_inst_imports
.iter()
.find(|inst| {
assert_eq!(inst.class.opcode, Op::ExtInstImport);
inst.operands[0].unwrap_literal_string() == &custom_insts::CUSTOM_EXT_INST_SET[..]
})
.map(|inst| inst.result_id.unwrap());
this.id_to_op_constant_operand.extend(
module
.types_global_values
.iter()
.filter(|inst| inst.class.opcode == Op::Constant)
.map(|inst| (inst.result_id.unwrap(), &inst.operands[0])),
);
let mut insts = module.debug_string_source.iter().peekable();
while let Some(inst) = insts.next() {
match inst.class.opcode {
@ -327,20 +354,40 @@ impl<'a> SpanRegenerator<'a> {
}
/// Extract the equivalent `SrcLocDecoration` from a debug instruction that
/// specifies some source location (currently only `OpLine` is supported).
/// specifies some source location (both the standard `OpLine`, and our own
/// custom instruction, i.e. `CustomInst::SetDebugSrcLoc`, are supported).
pub fn src_loc_from_debug_inst(&mut self, inst: &Instruction) -> Option<SrcLocDecoration<'a>> {
assert_eq!(inst.class.opcode, Op::Line);
let spv_debug_info = self
.spv_debug_info
.get_or_insert_with(|| SpvDebugInfo::collect(self.module));
let (file_id, line, col) = match inst.class.opcode {
Op::Line => (
inst.operands[0].unwrap_id_ref(),
inst.operands[1].unwrap_literal_int32(),
inst.operands[2].unwrap_literal_int32(),
),
Op::ExtInst
if Some(inst.operands[0].unwrap_id_ref())
== spv_debug_info.custom_ext_inst_set_import =>
{
match CustomInst::decode(inst) {
CustomInst::SetDebugSrcLoc { file, line, col } => (
file.unwrap_id_ref(),
spv_debug_info.id_to_op_constant_operand[&line.unwrap_id_ref()]
.unwrap_literal_int32(),
spv_debug_info.id_to_op_constant_operand[&col.unwrap_id_ref()]
.unwrap_literal_int32(),
),
custom_inst @ CustomInst::ClearDebugSrcLoc => {
unreachable!("src_loc_from_debug_inst({inst:?} => {custom_inst:?})")
}
}
}
_ => unreachable!("src_loc_from_debug_inst({inst:?})"),
};
self.spv_debug_info
.get_or_insert_with(|| SpvDebugInfo::collect(self.module))
spv_debug_info
.id_to_op_string
.get(&file_id)
.map(|&file_name| SrcLocDecoration {

View File

@ -0,0 +1,118 @@
//! SPIR-V (extended) instructions specific to `rustc_codegen_spirv`, produced
//! during the original codegen of a crate, and consumed by the `linker`.
use lazy_static::lazy_static;
use rspirv::dr::{Instruction, Operand};
use rspirv::spirv::Op;
use smallvec::SmallVec;
/// Prefix for `CUSTOM_EXT_INST_SET` (`OpExtInstImport` "instruction set" name),
/// without any of the disambiguating suffixes added for specific revisions.
///
/// This **should not** be changed (if possible), to ensure version mismatches
/// can be detected (i.e. starting with this prefix, but the full name differs).
///
/// See `CUSTOM_EXT_INST_SET`'s docs for further constraints on the full name.
pub const CUSTOM_EXT_INST_SET_PREFIX: &str = concat!("Rust.", env!("CARGO_PKG_NAME"), ".");
lazy_static! {
/// `OpExtInstImport` "instruction set" name for all Rust-GPU instructions.
///
/// These considerations are relevant to the specific choice of name:
/// * does *not* start with `NonSemantic.`, as:
/// * some custom instructions may need to be semantic
/// * these custom instructions are not meant for the final SPIR-V
/// (so no third-party support is *technically* required for them)
/// * `NonSemantic.` requires SPIR-V 1.6 (or `SPV_KHR_non_semantic_info`)
/// * always starts with `CUSTOM_EXT_INST_SET_PREFIX` (see also its docs),
/// regardless of Rust-GPU version or custom instruction set definition
/// * contains enough disambiguating information to avoid misinterpretation
/// if the definitions of the custom instructions have changed - this is
/// achieved by hashing the `SCHEMA` constant from `def_custom_insts!` below
pub static ref CUSTOM_EXT_INST_SET: String = {
const VER_MAJOR: &str = env!("CARGO_PKG_VERSION_MAJOR");
const VER_MINOR: &str = env!("CARGO_PKG_VERSION_MINOR");
const VER_PATCH: &str = env!("CARGO_PKG_VERSION_PATCH");
let schema_hash = {
use rustc_data_structures::stable_hasher::StableHasher;
use std::hash::Hash;
let mut hasher = StableHasher::new();
SCHEMA.hash(&mut hasher);
let (lo, hi) = hasher.finalize();
(lo as u128) | ((hi as u128) << 64)
};
format!("{CUSTOM_EXT_INST_SET_PREFIX}{VER_MAJOR}_{VER_MINOR}_{VER_PATCH}.{schema_hash:x}")
};
}
macro_rules! def_custom_insts {
($($num:literal => $name:ident $({ $($field:ident),+ $(,)? })?),+ $(,)?) => {
const SCHEMA: &[(u32, &str, &[&str])] = &[
$(($num, stringify!($name), &[$($(stringify!($field)),+)?])),+
];
#[repr(u32)]
#[derive(Copy, Clone, Debug, PartialEq, Eq)]
pub enum CustomOp { $($name = $num),+ }
impl CustomOp {
pub fn decode(i: u32) -> Self {
match i {
$($num => Self::$name,)+
_ => unreachable!("{i} is not a valid custom instruction number"),
}
}
pub fn decode_from_ext_inst(inst: &Instruction) -> Self {
assert_eq!(inst.class.opcode, Op::ExtInst);
Self::decode(inst.operands[1].unwrap_literal_ext_inst_integer())
}
pub fn with_operands<T: Clone>(self, operands: &[T]) -> CustomInst<T> {
match self {
$(Self::$name => match operands {
[$($($field),+)?] => CustomInst::$name $({ $($field: $field.clone()),+ })?,
_ => unreachable!("{self:?} does not have the right number of operands"),
}),+
}
}
}
#[derive(Copy, Clone, Debug)]
pub enum CustomInst<T> {
$($name $({ $($field: T),+ })?),+
}
impl<T> CustomInst<T> {
pub fn op(&self) -> CustomOp {
match *self {
$(Self::$name { .. } => CustomOp::$name),+
}
}
// HACK(eddyb) this should return an iterator, but that's too much effort.
pub fn into_operands(self) -> SmallVec<[T; 8]> {
match self {
$(Self::$name $({ $($field),+ })? => [$($($field),+)?].into_iter().collect()),+
}
}
}
impl CustomInst<Operand> {
pub fn decode(inst: &Instruction) -> Self {
CustomOp::decode_from_ext_inst(inst).with_operands(&inst.operands[2..])
}
}
}
}
def_custom_insts! {
// `OpLine` equivalent.
// FIXME(eddyb) improve on this, by adding more information.
0 => SetDebugSrcLoc { file, line, col },
// `OpNoLine` equivalent.
1 => ClearDebugSrcLoc,
}

View File

@ -73,6 +73,7 @@ mod builder;
mod builder_spirv;
mod codegen_cx;
mod custom_decorations;
mod custom_insts;
mod link;
mod linker;
mod spirv_type;

View File

@ -20,6 +20,7 @@ use std::borrow::Cow;
use crate::codegen_cx::SpirvMetadata;
use crate::custom_decorations::{CustomDecoration, SrcLocDecoration, ZombieDecoration};
use crate::custom_insts;
use either::Either;
use rspirv::binary::{Assemble, Consumer};
use rspirv::dr::{Block, Instruction, Loader, Module, ModuleHeader, Operand};
@ -56,6 +57,7 @@ pub struct Options {
pub dump_post_merge: Option<PathBuf>,
pub dump_post_split: Option<PathBuf>,
pub dump_spirt_passes: Option<PathBuf>,
pub spirt_keep_custom_debuginfo_in_dumps: bool,
pub specializer_debug: bool,
pub specializer_dump_instances: Option<PathBuf>,
pub print_all_zombie: bool,
@ -425,6 +427,15 @@ pub fn link(
.join(disambiguated_crate_name_for_dumps)
.with_extension("spirt");
// HACK(eddyb) unless requested otherwise, clean up the pretty-printed
// SPIR-T output by converting our custom extended instructions, to
// standard SPIR-V debuginfo (which SPIR-T knows how to pretty-print).
if !opts.spirt_keep_custom_debuginfo_in_dumps {
for (_, module) in &mut per_pass_module_for_dumping {
spirt_passes::debuginfo::convert_custom_debuginfo_to_spv(module);
}
}
let plan = spirt::print::Plan::for_versions(
&cx,
per_pass_module_for_dumping
@ -446,10 +457,13 @@ pub fn link(
}
// NOTE(eddyb) this is late so that `--dump-spirt-passes` is processed,
// even/especially when error were reported, but lifting to SPIR-V is
// even/especially when errors were reported, but lifting to SPIR-V is
// skipped (since it could very well fail due to reported errors).
report_diagnostics_result?;
// Replace our custom debuginfo instructions just before lifting to SPIR-V.
spirt_passes::debuginfo::convert_custom_debuginfo_to_spv(&mut module);
let spv_words = {
let _timer = sess.timer("spirt::Module::lift_to_spv_module_emitter");
module.lift_to_spv_module_emitter().unwrap().words
@ -462,6 +476,26 @@ pub fn link(
};
}
// Ensure that no references remain, to our custom "extended instruction set".
for inst in &output.ext_inst_imports {
assert_eq!(inst.class.opcode, Op::ExtInstImport);
let ext_inst_set = inst.operands[0].unwrap_literal_string();
if ext_inst_set.starts_with(custom_insts::CUSTOM_EXT_INST_SET_PREFIX) {
let expected = &custom_insts::CUSTOM_EXT_INST_SET[..];
if ext_inst_set == expected {
return Err(sess.err(format!(
"`OpExtInstImport {ext_inst_set:?}` should not have been \
left around after SPIR-T passes"
)));
} else {
return Err(sess.err(format!(
"unsupported `OpExtInstImport {ext_inst_set:?}`
(expected {expected:?} name - version mismatch?)"
)));
}
}
}
// FIXME(eddyb) rewrite these passes to SPIR-T ones, so we don't have to
// parse the output of `spirt::spv::lift` back into `rspirv` - also, for
// multi-module, it's much simpler with SPIR-T, just replace `module.exports`

View File

@ -143,6 +143,7 @@ pub fn name_variables_pass(module: &mut Module) {
module
.debug_names
.retain(|inst| variables.contains(&inst.operands[0].unwrap_id_ref()));
// FIXME(eddyb) why does this remove `OpLine` instructions?
module
.types_global_values
.retain(|inst| inst.class.opcode != Op::Line);

View File

@ -0,0 +1,150 @@
//! SPIR-T passes related to debuginfo.
use crate::custom_insts::{self, CustomInst, CustomOp};
use rustc_data_structures::fx::FxIndexSet;
use smallvec::SmallVec;
use spirt::transform::{InnerInPlaceTransform, Transformer};
use spirt::visit::InnerVisit;
use spirt::{
spv, Attr, AttrSetDef, ConstCtor, Context, ControlNode, ControlNodeKind, DataInstKind,
InternedStr, Module, OrdAssertEq, Value,
};
/// Replace our custom extended instruction debuginfo with standard SPIR-V ones.
//
// FIXME(eddyb) also handle `SrcLocDecoration`s (when `rspirv` isn't used on the
// SPIR-V output of `spirt::spv::lift`, as it's lossy wrt `OpLine`).
pub fn convert_custom_debuginfo_to_spv(module: &mut Module) {
let cx = &module.cx();
// FIXME(eddyb) reuse this collection work in some kind of "pass manager".
let all_funcs = {
let mut collector = super::ReachableUseCollector {
cx,
module,
seen_types: FxIndexSet::default(),
seen_consts: FxIndexSet::default(),
seen_global_vars: FxIndexSet::default(),
seen_funcs: FxIndexSet::default(),
};
for (export_key, &exportee) in &module.exports {
export_key.inner_visit_with(&mut collector);
exportee.inner_visit_with(&mut collector);
}
collector.seen_funcs
};
let mut transformer = CustomDebuginfoToSpv {
cx,
wk: &super::SpvSpecWithExtras::get().well_known,
custom_ext_inst_set: cx.intern(&custom_insts::CUSTOM_EXT_INST_SET[..]),
};
for func in all_funcs {
transformer.in_place_transform_func_decl(&mut module.funcs[func]);
}
}
struct CustomDebuginfoToSpv<'a> {
cx: &'a Context,
wk: &'static super::SpvWellKnownWithExtras,
/// Interned name for our custom "extended instruction set"
/// (see `crate::custom_insts` for more details).
custom_ext_inst_set: InternedStr,
}
impl Transformer for CustomDebuginfoToSpv<'_> {
fn in_place_transform_control_node_def(
&mut self,
mut func_at_control_node: spirt::func_at::FuncAtMut<'_, ControlNode>,
) {
// HACK(eddyb) this relies on the fact that `ControlNodeKind::Block` maps
// to one original SPIR-V block, which may not necessarily be true, and
// steps should be taken elsewhere to explicitly unset debuginfo, instead
// of relying on the end of a SPIR-V block implicitly unsetting it all.
// NOTE(eddyb) allowing debuginfo to apply *outside* of a `Block` could
// be useful in allowing *some* structured control-flow to have debuginfo,
// but that would likely require more work on the SPIR-T side.
if let ControlNodeKind::Block { mut insts } = func_at_control_node.reborrow().def().kind {
let mut current_file_line_col = None;
// HACK(eddyb) buffering the `DataInst`s to remove from this block,
// as iterating and modifying a list at the same time isn't supported.
let mut insts_to_remove = SmallVec::<[_; 8]>::new();
let mut func_at_inst_iter = func_at_control_node.reborrow().at(insts).into_iter();
while let Some(func_at_inst) = func_at_inst_iter.next() {
let inst = func_at_inst.position;
let data_inst_def = func_at_inst.def();
// FIXME(eddyb) deduplicate with `spirt_passes::diagnostics`.
if let DataInstKind::SpvExtInst {
ext_set,
inst: ext_inst,
} = data_inst_def.kind
{
if ext_set == self.custom_ext_inst_set {
match CustomOp::decode(ext_inst).with_operands(&data_inst_def.inputs) {
CustomInst::SetDebugSrcLoc { file, line, col } => {
let const_ctor = |v: Value| match v {
Value::Const(ct) => &self.cx[ct].ctor,
_ => unreachable!(),
};
let const_str = |v: Value| match const_ctor(v) {
&ConstCtor::SpvStringLiteralForExtInst(s) => s,
_ => unreachable!(),
};
let const_u32 = |v: Value| match const_ctor(v) {
ConstCtor::SpvInst(spv_inst) => {
assert!(spv_inst.opcode == self.wk.OpConstant);
match spv_inst.imms[..] {
[spv::Imm::Short(_, x)] => x,
_ => unreachable!(),
}
}
_ => unreachable!(),
};
current_file_line_col =
Some((const_str(file), const_u32(line), const_u32(col)));
insts_to_remove.push(inst);
continue;
}
CustomInst::ClearDebugSrcLoc => {
current_file_line_col = None;
insts_to_remove.push(inst);
continue;
}
}
}
}
// Add/remove the equivalent `Attr::SpvDebugLine` attribute.
// FIXME(eddyb) this could use more caching.
data_inst_def.attrs = self.cx.intern(AttrSetDef {
attrs: self.cx[data_inst_def.attrs]
.attrs
.iter()
.filter(|attr| !matches!(attr, Attr::SpvDebugLine { .. }))
.cloned()
.chain(
current_file_line_col.map(|(file, line, col)| Attr::SpvDebugLine {
file_path: OrdAssertEq(file),
line,
col,
}),
)
.collect(),
});
}
// Finally remove the `DataInst`s buffered for removal earlier.
for inst in insts_to_remove {
insts.remove(inst, func_at_control_node.data_insts);
}
func_at_control_node.reborrow().def().kind = ControlNodeKind::Block { insts };
}
func_at_control_node.inner_in_place_transform_with(self);
}
}

View File

@ -1,13 +1,18 @@
use crate::custom_decorations::{CustomDecoration, SpanRegenerator, SrcLocDecoration, ZombieDecoration};
use crate::custom_decorations::{
CustomDecoration, SpanRegenerator, SrcLocDecoration, ZombieDecoration,
};
use crate::custom_insts::{self, CustomInst, CustomOp};
use rustc_data_structures::fx::FxIndexSet;
use rustc_errors::DiagnosticBuilder;
use rustc_session::Session;
use rustc_span::{Span, DUMMY_SP};
use smallvec::SmallVec;
use spirt::func_at::FuncAt;
use spirt::visit::{InnerVisit, Visitor};
use spirt::{
spv, Attr, AttrSet, AttrSetDef, Const, Context, DataInstDef, DataInstKind, Diag, DiagLevel,
ExportKey, Exportee, Func, GlobalVar, Module, Type,
spv, Attr, AttrSet, AttrSetDef, Const, ConstCtor, Context, ControlNode, ControlNodeKind,
DataInstDef, DataInstKind, Diag, DiagLevel, ExportKey, Exportee, Func, GlobalVar, InternedStr,
Module, Type, Value,
};
use std::marker::PhantomData;
use std::{mem, str};
@ -24,6 +29,8 @@ pub(crate) fn report_diagnostics(
linker_options,
cx,
custom_ext_inst_set: cx.intern(&custom_insts::CUSTOM_EXT_INST_SET[..]),
module,
seen_attrs: FxIndexSet::default(),
@ -37,14 +44,15 @@ pub(crate) fn report_diagnostics(
overall_result: Ok(()),
any_spirt_bugs: false,
};
for (export_key, &exportee) in &module.exports {
for (export_key, exportee) in &module.exports {
assert_eq!(reporter.use_stack.len(), 0);
if let Exportee::Func(func) = exportee {
if let &Exportee::Func(func) = exportee {
let func_decl = &module.funcs[func];
reporter.use_stack.push(UseOrigin::IntraFunc {
func_attrs: func_decl.attrs,
func_export_key: Some(export_key),
last_debug_src_loc_inst: None,
inst_attrs: AttrSet::default(),
origin: IntraFuncUseOrigin::Other,
});
@ -149,6 +157,11 @@ struct DiagnosticReporter<'a> {
linker_options: &'a crate::linker::Options,
cx: &'a Context,
/// Interned name for our custom "extended instruction set"
/// (see `crate::custom_insts` for more details).
custom_ext_inst_set: InternedStr,
module: &'a Module,
seen_attrs: FxIndexSet<AttrSet>,
@ -172,6 +185,10 @@ enum UseOrigin<'a> {
func_attrs: AttrSet,
func_export_key: Option<&'a ExportKey>,
/// Active debug "source location" instruction at the time of the use, if any
/// (only `CustomInst::SetDebugSrcLoc` is supported).
last_debug_src_loc_inst: Option<&'a DataInstDef>,
inst_attrs: AttrSet,
origin: IntraFuncUseOrigin,
},
@ -184,7 +201,7 @@ enum IntraFuncUseOrigin {
impl UseOrigin<'_> {
fn to_rustc_span(&self, cx: &Context, span_regen: &mut SpanRegenerator<'_>) -> Option<Span> {
let mut from_attrs = |attrs: AttrSet| {
let from_attrs = |attrs: AttrSet, span_regen: &mut SpanRegenerator<'_>| {
let attrs_def = &cx[attrs];
attrs_def
.attrs
@ -208,12 +225,62 @@ impl UseOrigin<'_> {
})
};
match *self {
Self::Global { attrs, .. } => from_attrs(attrs),
Self::Global { attrs, .. } => from_attrs(attrs, span_regen),
Self::IntraFunc {
func_attrs,
last_debug_src_loc_inst,
inst_attrs,
..
} => from_attrs(inst_attrs).or_else(|| from_attrs(func_attrs)),
} => from_attrs(inst_attrs, span_regen)
.or_else(|| {
let debug_inst_def = last_debug_src_loc_inst?;
let wk = &super::SpvSpecWithExtras::get().well_known;
// FIXME(eddyb) deduplicate with `spirt_passes::diagnostics`.
let custom_op = match debug_inst_def.kind {
DataInstKind::SpvExtInst {
ext_set,
inst: ext_inst,
} => {
// FIXME(eddyb) inefficient (ideally the `InternedStr`
// shoudl be available), but this is the error case.
assert_eq!(&cx[ext_set], &custom_insts::CUSTOM_EXT_INST_SET[..]);
CustomOp::decode(ext_inst)
}
_ => unreachable!(),
};
let (file, line, col) = match custom_op.with_operands(&debug_inst_def.inputs) {
CustomInst::SetDebugSrcLoc { file, line, col } => (file, line, col),
CustomInst::ClearDebugSrcLoc => unreachable!(),
};
let const_ctor = |v: Value| match v {
Value::Const(ct) => &cx[ct].ctor,
_ => unreachable!(),
};
let const_str = |v: Value| match const_ctor(v) {
&ConstCtor::SpvStringLiteralForExtInst(s) => s,
_ => unreachable!(),
};
let const_u32 = |v: Value| match const_ctor(v) {
ConstCtor::SpvInst(spv_inst) => {
assert!(spv_inst.opcode == wk.OpConstant);
match spv_inst.imms[..] {
[spv::Imm::Short(_, x)] => x,
_ => unreachable!(),
}
}
_ => unreachable!(),
};
span_regen.src_loc_to_rustc(SrcLocDecoration {
file_name: &cx[const_str(file)],
line: const_u32(line),
col: const_u32(col),
})
})
.or_else(|| from_attrs(func_attrs, span_regen)),
}
}
@ -246,6 +313,7 @@ impl UseOrigin<'_> {
Self::IntraFunc {
func_attrs,
func_export_key,
last_debug_src_loc_inst: _,
inst_attrs: _,
origin,
} => {
@ -375,7 +443,7 @@ impl DiagnosticReporter<'_> {
}
}
impl Visitor<'_> for DiagnosticReporter<'_> {
impl<'a> Visitor<'a> for DiagnosticReporter<'a> {
fn visit_attr_set_use(&mut self, attrs: AttrSet) {
// HACK(eddyb) this avoids reporting the same diagnostics more than once.
if self.seen_attrs.insert(attrs) {
@ -424,6 +492,7 @@ impl Visitor<'_> for DiagnosticReporter<'_> {
self.use_stack.push(UseOrigin::IntraFunc {
func_attrs: func_decl.attrs,
func_export_key: None,
last_debug_src_loc_inst: None,
inst_attrs: AttrSet::default(),
origin: IntraFuncUseOrigin::Other,
});
@ -432,9 +501,54 @@ impl Visitor<'_> for DiagnosticReporter<'_> {
}
}
fn visit_data_inst_def(&mut self, data_inst_def: &DataInstDef) {
fn visit_control_node_def(&mut self, func_at_control_node: FuncAt<'a, ControlNode>) {
func_at_control_node.inner_visit_with(self);
// HACK(eddyb) this relies on the fact that `ControlNodeKind::Block` maps
// to one original SPIR-V block, which may not necessarily be true, and
// steps should be taken elsewhere to explicitly unset debuginfo, instead
// of relying on the end of a SPIR-V block implicitly unsetting it all.
// NOTE(eddyb) allowing debuginfo to apply *outside* of a `Block` could
// be useful in allowing *some* structured control-flow to have debuginfo,
// but that would likely require more work on the SPIR-T side.
if let ControlNodeKind::Block { .. } = func_at_control_node.def().kind {
match self.use_stack.last_mut() {
Some(UseOrigin::IntraFunc {
last_debug_src_loc_inst,
..
}) => *last_debug_src_loc_inst = None,
_ => unreachable!(),
}
}
}
fn visit_data_inst_def(&mut self, data_inst_def: &'a DataInstDef) {
match self.use_stack.last_mut() {
Some(UseOrigin::IntraFunc { inst_attrs, .. }) => *inst_attrs = data_inst_def.attrs,
Some(UseOrigin::IntraFunc {
inst_attrs,
last_debug_src_loc_inst,
..
}) => {
*inst_attrs = data_inst_def.attrs;
// FIXME(eddyb) deduplicate with `spirt_passes::debuginfo`.
if let DataInstKind::SpvExtInst {
ext_set,
inst: ext_inst,
} = data_inst_def.kind
{
if ext_set == self.custom_ext_inst_set {
match CustomOp::decode(ext_inst) {
CustomOp::SetDebugSrcLoc => {
*last_debug_src_loc_inst = Some(data_inst_def);
}
CustomOp::ClearDebugSrcLoc => {
*last_debug_src_loc_inst = None;
}
}
}
}
}
_ => unreachable!(),
}

View File

@ -1,5 +1,6 @@
//! SPIR-T pass infrastructure and supporting utilities.
pub(crate) mod debuginfo;
pub(crate) mod diagnostics;
mod fuse_selects;
mod reduce;

View File

@ -2,6 +2,7 @@
use super::{get_name, get_names};
use crate::custom_decorations::{CustomDecoration, SpanRegenerator, ZombieDecoration};
use crate::custom_insts::{self, CustomOp};
use rspirv::dr::{Instruction, Module, Operand};
use rspirv::spirv::{Op, Word};
use rustc_data_structures::fx::{FxHashMap, FxHashSet, FxIndexMap};
@ -27,7 +28,8 @@ enum ZombieKind<'a> {
struct ZombieUse<'a> {
used_zombie_id: Word,
/// Active `OpLine` instruction at the time of the use, if any.
/// Active debug "source location" instruction at the time of the use, if any
/// (both `OpLine` and `CustomInst::SetDebugSrcLoc` are supported).
use_debug_src_loc_inst: Option<&'a Instruction>,
origin: UseOrigin,
@ -40,6 +42,10 @@ enum UseOrigin {
}
struct Zombies<'a> {
/// ID of `OpExtInstImport` for our custom "extended instruction set",
/// if present (see `crate::custom_insts` for more details).
custom_ext_inst_set_import: Option<Word>,
id_to_zombie_kind: FxIndexMap<Word, ZombieKind<'a>>,
}
@ -70,6 +76,15 @@ impl<'a> Zombies<'a> {
match inst.class.opcode {
Op::Line => debug_src_loc_inst = Some(inst),
Op::NoLine => debug_src_loc_inst = None,
Op::ExtInst
if Some(inst.operands[0].unwrap_id_ref())
== self.custom_ext_inst_set_import =>
{
match CustomOp::decode_from_ext_inst(inst) {
CustomOp::SetDebugSrcLoc => debug_src_loc_inst = Some(inst),
CustomOp::ClearDebugSrcLoc => debug_src_loc_inst = None,
}
}
_ => {}
}
@ -113,6 +128,15 @@ impl<'a> Zombies<'a> {
Op::Line => debug_src_loc_inst = Some(inst),
// NOTE(eddyb) each block starts out with cleared debuginfo.
Op::Label | Op::NoLine => debug_src_loc_inst = None,
Op::ExtInst
if Some(inst.operands[0].unwrap_id_ref())
== self.custom_ext_inst_set_import =>
{
match CustomOp::decode_from_ext_inst(inst) {
CustomOp::SetDebugSrcLoc => debug_src_loc_inst = Some(inst),
CustomOp::ClearDebugSrcLoc => debug_src_loc_inst = None,
}
}
_ => {}
}
@ -301,6 +325,16 @@ pub fn report_and_remove_zombies(
module: &mut Module,
) -> super::Result<()> {
let mut zombies = Zombies {
// FIXME(eddyb) avoid repeating this across different passes/helpers.
custom_ext_inst_set_import: module
.ext_inst_imports
.iter()
.find(|inst| {
assert_eq!(inst.class.opcode, Op::ExtInstImport);
inst.operands[0].unwrap_literal_string() == &custom_insts::CUSTOM_EXT_INST_SET[..]
})
.map(|inst| inst.result_id.unwrap()),
id_to_zombie_kind: ZombieDecoration::decode_all(module)
.map(|(id, _)| (id, ZombieKind::Leaf))
.collect(),

View File

@ -191,3 +191,12 @@ _Note: passes that are not already enabled by default are considered experimenta
Dump the `SPIR-🇹` module across passes (i.e. all of the versions before/after each pass), as a combined report, to a pair of files (`.spirt` and `.spirt.html`) in `DIR`.
<sub>(the `.spirt.html` version of the report is the recommended form for viewing, as it uses tabling for versions, syntax-highlighting-like styling, and use->def linking)</sub>
### `--spirt-keep-custom-debuginfo-in-dumps`
When dumping (pretty-printed) `SPIR-🇹` (e.g. with `--dump-spirt-passes`), preserve
all the custom (Rust-GPU-specific) debuginfo instructions, instead of converting
them to the standard SPIR-V debuginfo (which `SPIR-🇹` pretty-prints specially).
The default (of performing that conversion) has prettier results, but is lossier
if you want to see all instructions exactly as e.g. `--spirt-passes` see them.