feat: Implement cast typechecks

This commit is contained in:
Shoyu Vanilla 2024-08-29 00:12:16 +09:00
parent 304e5f58cd
commit d186bdc617
25 changed files with 1614 additions and 93 deletions

View file

@ -14,6 +14,7 @@ use triomphe::Arc;
use crate::{ use crate::{
builtin_type::{BuiltinInt, BuiltinUint}, builtin_type::{BuiltinInt, BuiltinUint},
db::DefDatabase, db::DefDatabase,
hir::Expr,
item_tree::{ item_tree::{
AttrOwner, Field, FieldParent, FieldsShape, ItemTree, ModItem, RawVisibilityId, TreeId, AttrOwner, Field, FieldParent, FieldsShape, ItemTree, ModItem, RawVisibilityId, TreeId,
}, },
@ -317,6 +318,27 @@ impl EnumData {
_ => IntegerType::Pointer(true), _ => IntegerType::Pointer(true),
} }
} }
// [Adopted from rustc](https://github.com/rust-lang/rust/blob/bd53aa3bf7a24a70d763182303bd75e5fc51a9af/compiler/rustc_middle/src/ty/adt.rs#L446-L448)
pub fn is_payload_free(&self, db: &dyn DefDatabase) -> bool {
self.variants.iter().all(|(v, _)| {
// The condition check order is slightly modified from rustc
// to improve performance by early returning with relatively fast checks
let variant = &db.enum_variant_data(*v).variant_data;
if !variant.fields().is_empty() {
return false;
}
// The outer if condition is whether this variant has const ctor or not
if !matches!(variant.kind(), StructKind::Unit) {
let body = db.body((*v).into());
// A variant with explicit discriminant
if body.exprs[body.body_expr] != Expr::Missing {
return false;
}
}
true
})
}
} }
impl EnumVariantData { impl EnumVariantData {

View file

@ -186,7 +186,13 @@ fn floating_point() {
#[test] #[test]
fn casts() { fn casts() {
check_number(r#"const GOAL: usize = 12 as *const i32 as usize"#, 12); check_number(
r#"
//- minicore: sized
const GOAL: usize = 12 as *const i32 as usize
"#,
12,
);
check_number( check_number(
r#" r#"
//- minicore: coerce_unsized, index, slice //- minicore: coerce_unsized, index, slice
@ -204,7 +210,7 @@ fn casts() {
r#" r#"
//- minicore: coerce_unsized, index, slice //- minicore: coerce_unsized, index, slice
const GOAL: i16 = { const GOAL: i16 = {
let a = &mut 5; let a = &mut 5_i16;
let z = a as *mut _; let z = a as *mut _;
unsafe { *z } unsafe { *z }
}; };
@ -244,7 +250,13 @@ fn casts() {
"#, "#,
4, 4,
); );
check_number(r#"const GOAL: i32 = -12i8 as i32"#, -12); check_number(
r#"
//- minicore: sized
const GOAL: i32 = -12i8 as i32
"#,
-12,
);
} }
#[test] #[test]
@ -1911,6 +1923,7 @@ fn function_pointer() {
); );
check_number( check_number(
r#" r#"
//- minicore: sized
fn add2(x: u8) -> u8 { fn add2(x: u8) -> u8 {
x + 2 x + 2
} }
@ -2422,6 +2435,7 @@ fn statics() {
fn extern_weak_statics() { fn extern_weak_statics() {
check_number( check_number(
r#" r#"
//- minicore: sized
extern "C" { extern "C" {
#[linkage = "extern_weak"] #[linkage = "extern_weak"]
static __dso_handle: *mut u8; static __dso_handle: *mut u8;
@ -2716,6 +2730,7 @@ fn const_trait_assoc() {
); );
check_number( check_number(
r#" r#"
//- minicore: sized
struct S<T>(*mut T); struct S<T>(*mut T);
trait MySized: Sized { trait MySized: Sized {

View file

@ -311,6 +311,7 @@ fn saturating() {
fn allocator() { fn allocator() {
check_number( check_number(
r#" r#"
//- minicore: sized
extern "Rust" { extern "Rust" {
#[rustc_allocator] #[rustc_allocator]
fn __rust_alloc(size: usize, align: usize) -> *mut u8; fn __rust_alloc(size: usize, align: usize) -> *mut u8;

View file

@ -13,7 +13,7 @@
//! to certain types. To record this, we use the union-find implementation from //! to certain types. To record this, we use the union-find implementation from
//! the `ena` crate, which is extracted from rustc. //! the `ena` crate, which is extracted from rustc.
mod cast; pub(crate) mod cast;
pub(crate) mod closure; pub(crate) mod closure;
mod coerce; mod coerce;
mod expr; mod expr;
@ -76,7 +76,7 @@ pub use coerce::could_coerce;
#[allow(unreachable_pub)] #[allow(unreachable_pub)]
pub use unify::{could_unify, could_unify_deeply}; pub use unify::{could_unify, could_unify_deeply};
use cast::CastCheck; use cast::{CastCheck, CastError};
pub(crate) use closure::{CaptureKind, CapturedItem, CapturedItemWithoutTy}; pub(crate) use closure::{CaptureKind, CapturedItem, CapturedItemWithoutTy};
/// The entry point of type inference. /// The entry point of type inference.
@ -254,6 +254,16 @@ pub enum InferenceDiagnostic {
expr: ExprId, expr: ExprId,
expected: Ty, expected: Ty,
}, },
CastToUnsized {
expr: ExprId,
cast_ty: Ty,
},
InvalidCast {
expr: ExprId,
error: CastError,
expr_ty: Ty,
cast_ty: Ty,
},
} }
/// A mismatch between an expected and an inferred type. /// A mismatch between an expected and an inferred type.
@ -456,6 +466,7 @@ pub struct InferenceResult {
pub(crate) closure_info: FxHashMap<ClosureId, (Vec<CapturedItem>, FnTrait)>, pub(crate) closure_info: FxHashMap<ClosureId, (Vec<CapturedItem>, FnTrait)>,
// FIXME: remove this field // FIXME: remove this field
pub mutated_bindings_in_closure: FxHashSet<BindingId>, pub mutated_bindings_in_closure: FxHashSet<BindingId>,
pub coercion_casts: FxHashSet<ExprId>,
} }
impl InferenceResult { impl InferenceResult {
@ -666,7 +677,7 @@ impl<'a> InferenceContext<'a> {
let InferenceContext { let InferenceContext {
mut table, mut table,
mut result, mut result,
deferred_cast_checks, mut deferred_cast_checks,
tuple_field_accesses_rev, tuple_field_accesses_rev,
.. ..
} = self; } = self;
@ -695,6 +706,7 @@ impl<'a> InferenceContext<'a> {
closure_info: _, closure_info: _,
mutated_bindings_in_closure: _, mutated_bindings_in_closure: _,
tuple_field_access_types: _, tuple_field_access_types: _,
coercion_casts,
} = &mut result; } = &mut result;
table.fallback_if_possible(); table.fallback_if_possible();
@ -702,8 +714,18 @@ impl<'a> InferenceContext<'a> {
// Comment from rustc: // Comment from rustc:
// Even though coercion casts provide type hints, we check casts after fallback for // Even though coercion casts provide type hints, we check casts after fallback for
// backwards compatibility. This makes fallback a stronger type hint than a cast coercion. // backwards compatibility. This makes fallback a stronger type hint than a cast coercion.
for cast in deferred_cast_checks { let mut apply_adjustments = |expr, adj| {
cast.check(&mut table); expr_adjustments.insert(expr, adj);
};
let mut set_coercion_cast = |expr| {
coercion_casts.insert(expr);
};
for cast in deferred_cast_checks.iter_mut() {
if let Err(diag) =
cast.check(&mut table, &mut apply_adjustments, &mut set_coercion_cast)
{
diagnostics.push(diag);
}
} }
// FIXME resolve obligations as well (use Guidance if necessary) // FIXME resolve obligations as well (use Guidance if necessary)

View file

@ -1,47 +1,412 @@
//! Type cast logic. Basically coercion + additional casts. //! Type cast logic. Basically coercion + additional casts.
use crate::{infer::unify::InferenceTable, Interner, Ty, TyExt, TyKind}; use chalk_ir::{Mutability, Scalar, TyVariableKind, UintTy};
use hir_def::{hir::ExprId, AdtId};
use stdx::never;
use crate::{
infer::unify::InferenceTable, Adjustment, Binders, DynTy, InferenceDiagnostic, Interner,
PlaceholderIndex, QuantifiedWhereClauses, Ty, TyExt, TyKind, TypeFlags, WhereClause,
};
#[derive(Debug)]
pub(crate) enum Int {
I,
U(UintTy),
Bool,
Char,
CEnum,
InferenceVar,
}
#[derive(Debug)]
pub(crate) enum CastTy {
Int(Int),
Float,
FnPtr,
Ptr(Ty, Mutability),
// `DynStar` is Not supported yet in r-a
}
impl CastTy {
pub(crate) fn from_ty(table: &mut InferenceTable<'_>, t: &Ty) -> Option<Self> {
match t.kind(Interner) {
TyKind::Scalar(Scalar::Bool) => Some(Self::Int(Int::Bool)),
TyKind::Scalar(Scalar::Char) => Some(Self::Int(Int::Char)),
TyKind::Scalar(Scalar::Int(_)) => Some(Self::Int(Int::I)),
TyKind::Scalar(Scalar::Uint(it)) => Some(Self::Int(Int::U(*it))),
TyKind::InferenceVar(_, TyVariableKind::Integer) => Some(Self::Int(Int::InferenceVar)),
TyKind::InferenceVar(_, TyVariableKind::Float) => Some(Self::Float),
TyKind::Scalar(Scalar::Float(_)) => Some(Self::Float),
TyKind::Adt(..) => {
let (AdtId::EnumId(id), _) = t.as_adt()? else {
return None;
};
let enum_data = table.db.enum_data(id);
if enum_data.is_payload_free(table.db.upcast()) {
Some(Self::Int(Int::CEnum))
} else {
None
}
}
TyKind::Raw(m, ty) => Some(Self::Ptr(table.resolve_ty_shallow(ty), *m)),
TyKind::Function(_) => Some(Self::FnPtr),
_ => None,
}
}
}
#[derive(Debug, PartialEq, Eq, Clone, Copy)]
pub enum CastError {
Unknown,
CastToBool,
CastToChar,
DifferingKinds,
SizedUnsizedCast,
IllegalCast,
IntToFatCast,
NeedDeref,
NeedViaPtr,
NeedViaThinPtr,
NeedViaInt,
NonScalar,
UnknownCastPtrKind,
UnknownExprPtrKind,
}
impl CastError {
fn into_diagnostic(self, expr: ExprId, expr_ty: Ty, cast_ty: Ty) -> InferenceDiagnostic {
InferenceDiagnostic::InvalidCast { expr, error: self, expr_ty, cast_ty }
}
}
#[derive(Clone, Debug)] #[derive(Clone, Debug)]
pub(super) struct CastCheck { pub(super) struct CastCheck {
expr: ExprId,
source_expr: ExprId,
expr_ty: Ty, expr_ty: Ty,
cast_ty: Ty, cast_ty: Ty,
} }
impl CastCheck { impl CastCheck {
pub(super) fn new(expr_ty: Ty, cast_ty: Ty) -> Self { pub(super) fn new(expr: ExprId, source_expr: ExprId, expr_ty: Ty, cast_ty: Ty) -> Self {
Self { expr_ty, cast_ty } Self { expr, source_expr, expr_ty, cast_ty }
} }
pub(super) fn check(self, table: &mut InferenceTable<'_>) { pub(super) fn check<F, G>(
// FIXME: This function currently only implements the bits that influence the type &mut self,
// inference. We should return the adjustments on success and report diagnostics on error. table: &mut InferenceTable<'_>,
let expr_ty = table.resolve_ty_shallow(&self.expr_ty); apply_adjustments: &mut F,
let cast_ty = table.resolve_ty_shallow(&self.cast_ty); set_coercion_cast: &mut G,
) -> Result<(), InferenceDiagnostic>
where
F: FnMut(ExprId, Vec<Adjustment>),
G: FnMut(ExprId),
{
table.resolve_obligations_as_possible();
self.expr_ty = table.resolve_ty_shallow(&self.expr_ty);
self.cast_ty = table.resolve_ty_shallow(&self.cast_ty);
if table.coerce(&expr_ty, &cast_ty).is_ok() { if self.expr_ty.contains_unknown() || self.cast_ty.contains_unknown() {
return; return Ok(());
} }
if check_ref_to_ptr_cast(expr_ty, cast_ty, table) { if !self.cast_ty.data(Interner).flags.contains(TypeFlags::HAS_TY_INFER)
// Note that this type of cast is actually split into a coercion to a && !table.is_sized(&self.cast_ty)
// pointer type and a cast: {
// &[T; N] -> *[T; N] -> *T return Err(InferenceDiagnostic::CastToUnsized {
expr: self.expr,
cast_ty: self.cast_ty.clone(),
});
} }
// FIXME: Check other kinds of non-coercion casts and report error if any? if let Ok((adj, _)) = table.coerce(&self.expr_ty, &self.cast_ty) {
apply_adjustments(self.source_expr, adj);
set_coercion_cast(self.source_expr);
return Ok(());
}
self.do_check(table, apply_adjustments)
.map_err(|e| e.into_diagnostic(self.expr, self.expr_ty.clone(), self.cast_ty.clone()))
}
fn do_check<F>(
&self,
table: &mut InferenceTable<'_>,
apply_adjustments: &mut F,
) -> Result<(), CastError>
where
F: FnMut(ExprId, Vec<Adjustment>),
{
let (t_from, t_cast) =
match (CastTy::from_ty(table, &self.expr_ty), CastTy::from_ty(table, &self.cast_ty)) {
(Some(t_from), Some(t_cast)) => (t_from, t_cast),
(None, Some(t_cast)) => match self.expr_ty.kind(Interner) {
TyKind::FnDef(..) => {
let sig = self.expr_ty.callable_sig(table.db).expect("FnDef had no sig");
let sig = table.normalize_associated_types_in(sig);
let fn_ptr = TyKind::Function(sig.to_fn_ptr()).intern(Interner);
if let Ok((adj, _)) = table.coerce(&self.expr_ty, &fn_ptr) {
apply_adjustments(self.source_expr, adj);
} else {
return Err(CastError::IllegalCast);
}
(CastTy::FnPtr, t_cast)
}
TyKind::Ref(mutbl, _, inner_ty) => {
let inner_ty = table.resolve_ty_shallow(inner_ty);
return match t_cast {
CastTy::Int(_) | CastTy::Float => match inner_ty.kind(Interner) {
TyKind::Scalar(
Scalar::Int(_) | Scalar::Uint(_) | Scalar::Float(_),
)
| TyKind::InferenceVar(
_,
TyVariableKind::Integer | TyVariableKind::Float,
) => Err(CastError::NeedDeref),
_ => Err(CastError::NeedViaPtr),
},
// array-ptr-cast
CastTy::Ptr(t, m) => {
let t = table.resolve_ty_shallow(&t);
if !table.is_sized(&t) {
return Err(CastError::IllegalCast);
}
self.check_ref_cast(
table,
&inner_ty,
*mutbl,
&t,
m,
apply_adjustments,
)
}
_ => Err(CastError::NonScalar),
};
}
_ => return Err(CastError::NonScalar),
},
_ => return Err(CastError::NonScalar),
};
// rustc checks whether the `expr_ty` is foreign adt with `non_exhaustive` sym
match (t_from, t_cast) {
(_, CastTy::Int(Int::CEnum) | CastTy::FnPtr) => Err(CastError::NonScalar),
(_, CastTy::Int(Int::Bool)) => Err(CastError::CastToBool),
(CastTy::Int(Int::U(UintTy::U8)), CastTy::Int(Int::Char)) => Ok(()),
(_, CastTy::Int(Int::Char)) => Err(CastError::CastToChar),
(CastTy::Int(Int::Bool | Int::CEnum | Int::Char), CastTy::Float) => {
Err(CastError::NeedViaInt)
}
(CastTy::Int(Int::Bool | Int::CEnum | Int::Char) | CastTy::Float, CastTy::Ptr(..))
| (CastTy::Ptr(..) | CastTy::FnPtr, CastTy::Float) => Err(CastError::IllegalCast),
(CastTy::Ptr(src, _), CastTy::Ptr(dst, _)) => {
self.check_ptr_ptr_cast(table, &src, &dst)
}
(CastTy::Ptr(src, _), CastTy::Int(_)) => self.check_ptr_addr_cast(table, &src),
(CastTy::Int(_), CastTy::Ptr(dst, _)) => self.check_addr_ptr_cast(table, &dst),
(CastTy::FnPtr, CastTy::Ptr(dst, _)) => self.check_fptr_ptr_cast(table, &dst),
(CastTy::Int(Int::CEnum), CastTy::Int(_)) => Ok(()),
(CastTy::Int(Int::Char | Int::Bool), CastTy::Int(_)) => Ok(()),
(CastTy::Int(_) | CastTy::Float, CastTy::Int(_) | CastTy::Float) => Ok(()),
(CastTy::FnPtr, CastTy::Int(_)) => Ok(()),
}
}
fn check_ref_cast<F>(
&self,
table: &mut InferenceTable<'_>,
t_expr: &Ty,
m_expr: Mutability,
t_cast: &Ty,
m_cast: Mutability,
apply_adjustments: &mut F,
) -> Result<(), CastError>
where
F: FnMut(ExprId, Vec<Adjustment>),
{
// Mutability order is opposite to rustc. `Mut < Not`
if m_expr <= m_cast {
if let TyKind::Array(ety, _) = t_expr.kind(Interner) {
// Coerce to a raw pointer so that we generate RawPtr in MIR.
let array_ptr_type = TyKind::Raw(m_expr, t_expr.clone()).intern(Interner);
if let Ok((adj, _)) = table.coerce(&self.expr_ty, &array_ptr_type) {
apply_adjustments(self.source_expr, adj);
} else {
never!(
"could not cast from reference to array to pointer to array ({:?} to {:?})",
self.expr_ty,
array_ptr_type
);
}
// This is a less strict condition than rustc's `demand_eqtype`,
// but false negative is better than false positive
if table.coerce(ety, t_cast).is_ok() {
return Ok(());
}
}
}
Err(CastError::IllegalCast)
}
fn check_ptr_ptr_cast(
&self,
table: &mut InferenceTable<'_>,
src: &Ty,
dst: &Ty,
) -> Result<(), CastError> {
let src_kind = pointer_kind(src, table).map_err(|_| CastError::Unknown)?;
let dst_kind = pointer_kind(dst, table).map_err(|_| CastError::Unknown)?;
match (src_kind, dst_kind) {
(Some(PointerKind::Error), _) | (_, Some(PointerKind::Error)) => Ok(()),
(_, None) => Err(CastError::UnknownCastPtrKind),
(_, Some(PointerKind::Thin)) => Ok(()),
(None, _) => Err(CastError::UnknownExprPtrKind),
(Some(PointerKind::Thin), _) => Err(CastError::SizedUnsizedCast),
(Some(PointerKind::VTable(src_tty)), Some(PointerKind::VTable(dst_tty))) => {
let principal = |tty: &Binders<QuantifiedWhereClauses>| {
tty.skip_binders().as_slice(Interner).first().and_then(|pred| {
if let WhereClause::Implemented(tr) = pred.skip_binders() {
Some(tr.trait_id)
} else {
None
}
})
};
match (principal(&src_tty), principal(&dst_tty)) {
(Some(src_principal), Some(dst_principal)) => {
if src_principal == dst_principal {
return Ok(());
}
let src_principal =
table.db.trait_datum(table.trait_env.krate, src_principal);
let dst_principal =
table.db.trait_datum(table.trait_env.krate, dst_principal);
if src_principal.is_auto_trait() && dst_principal.is_auto_trait() {
Ok(())
} else {
Err(CastError::DifferingKinds)
}
}
_ => Err(CastError::Unknown),
}
}
(Some(src_kind), Some(dst_kind)) if src_kind == dst_kind => Ok(()),
(_, _) => Err(CastError::DifferingKinds),
}
}
fn check_ptr_addr_cast(
&self,
table: &mut InferenceTable<'_>,
expr_ty: &Ty,
) -> Result<(), CastError> {
match pointer_kind(expr_ty, table).map_err(|_| CastError::Unknown)? {
None => Err(CastError::UnknownExprPtrKind),
Some(PointerKind::Error) => Ok(()),
Some(PointerKind::Thin) => Ok(()),
_ => Err(CastError::NeedViaThinPtr),
}
}
fn check_addr_ptr_cast(
&self,
table: &mut InferenceTable<'_>,
cast_ty: &Ty,
) -> Result<(), CastError> {
match pointer_kind(cast_ty, table).map_err(|_| CastError::Unknown)? {
None => Err(CastError::UnknownCastPtrKind),
Some(PointerKind::Error) => Ok(()),
Some(PointerKind::Thin) => Ok(()),
Some(PointerKind::VTable(_)) => Err(CastError::IntToFatCast),
Some(PointerKind::Length) => Err(CastError::IntToFatCast),
Some(PointerKind::OfAlias | PointerKind::OfParam(_)) => Err(CastError::IntToFatCast),
}
}
fn check_fptr_ptr_cast(
&self,
table: &mut InferenceTable<'_>,
cast_ty: &Ty,
) -> Result<(), CastError> {
match pointer_kind(cast_ty, table).map_err(|_| CastError::Unknown)? {
None => Err(CastError::UnknownCastPtrKind),
Some(PointerKind::Error) => Ok(()),
Some(PointerKind::Thin) => Ok(()),
_ => Err(CastError::IllegalCast),
}
} }
} }
fn check_ref_to_ptr_cast(expr_ty: Ty, cast_ty: Ty, table: &mut InferenceTable<'_>) -> bool { #[derive(PartialEq, Eq)]
let Some((expr_inner_ty, _, _)) = expr_ty.as_reference() else { enum PointerKind {
return false; // thin pointer
}; Thin,
let Some((cast_inner_ty, _)) = cast_ty.as_raw_ptr() else { // trait object
return false; VTable(Binders<QuantifiedWhereClauses>),
}; // slice
let TyKind::Array(expr_elt_ty, _) = expr_inner_ty.kind(Interner) else { Length,
return false; OfAlias,
}; OfParam(PlaceholderIndex),
table.coerce(expr_elt_ty, cast_inner_ty).is_ok() Error,
}
fn pointer_kind(ty: &Ty, table: &mut InferenceTable<'_>) -> Result<Option<PointerKind>, ()> {
let ty = table.resolve_ty_shallow(ty);
if table.is_sized(&ty) {
return Ok(Some(PointerKind::Thin));
}
match ty.kind(Interner) {
TyKind::Slice(_) | TyKind::Str => Ok(Some(PointerKind::Length)),
TyKind::Dyn(DynTy { bounds, .. }) => Ok(Some(PointerKind::VTable(bounds.clone()))),
TyKind::Adt(chalk_ir::AdtId(id), subst) => {
let AdtId::StructId(id) = *id else {
never!("`{:?}` should be sized but is not?", ty);
return Err(());
};
let struct_data = table.db.struct_data(id);
if let Some((last_field, _)) = struct_data.variant_data.fields().iter().last() {
let last_field_ty =
table.db.field_types(id.into())[last_field].clone().substitute(Interner, subst);
pointer_kind(&last_field_ty, table)
} else {
Ok(Some(PointerKind::Thin))
}
}
TyKind::Tuple(_, subst) => {
match subst.iter(Interner).last().and_then(|arg| arg.ty(Interner)) {
None => Ok(Some(PointerKind::Thin)),
Some(ty) => pointer_kind(ty, table),
}
}
TyKind::Foreign(_) => Ok(Some(PointerKind::Thin)),
TyKind::Alias(_) | TyKind::AssociatedType(..) | TyKind::OpaqueType(..) => {
Ok(Some(PointerKind::OfAlias))
}
TyKind::Error => Ok(Some(PointerKind::Error)),
TyKind::Placeholder(idx) => Ok(Some(PointerKind::OfParam(*idx))),
TyKind::BoundVar(_) | TyKind::InferenceVar(..) => Ok(None),
TyKind::Scalar(_)
| TyKind::Array(..)
| TyKind::CoroutineWitness(..)
| TyKind::Raw(..)
| TyKind::Ref(..)
| TyKind::FnDef(..)
| TyKind::Function(_)
| TyKind::Closure(..)
| TyKind::Coroutine(..)
| TyKind::Never => {
never!("`{:?}` should be sized but is not?", ty);
Err(())
}
}
} }

View file

@ -610,7 +610,12 @@ impl InferenceContext<'_> {
Expr::Cast { expr, type_ref } => { Expr::Cast { expr, type_ref } => {
let cast_ty = self.make_ty(type_ref); let cast_ty = self.make_ty(type_ref);
let expr_ty = self.infer_expr(*expr, &Expectation::Castable(cast_ty.clone())); let expr_ty = self.infer_expr(*expr, &Expectation::Castable(cast_ty.clone()));
self.deferred_cast_checks.push(CastCheck::new(expr_ty, cast_ty.clone())); self.deferred_cast_checks.push(CastCheck::new(
tgt_expr,
*expr,
expr_ty,
cast_ty.clone(),
));
cast_ty cast_ty
} }
Expr::Ref { expr, rawness, mutability } => { Expr::Ref { expr, rawness, mutability } => {

View file

@ -9,6 +9,7 @@ use chalk_ir::{
use chalk_solve::infer::ParameterEnaVariableExt; use chalk_solve::infer::ParameterEnaVariableExt;
use either::Either; use either::Either;
use ena::unify::UnifyKey; use ena::unify::UnifyKey;
use hir_def::{lang_item::LangItem, AdtId};
use hir_expand::name::Name; use hir_expand::name::Name;
use intern::sym; use intern::sym;
use rustc_hash::FxHashMap; use rustc_hash::FxHashMap;
@ -21,7 +22,7 @@ use crate::{
to_chalk_trait_id, traits::FnTrait, AliasEq, AliasTy, BoundVar, Canonical, Const, ConstValue, to_chalk_trait_id, traits::FnTrait, AliasEq, AliasTy, BoundVar, Canonical, Const, ConstValue,
DebruijnIndex, DomainGoal, GenericArg, GenericArgData, Goal, GoalData, Guidance, InEnvironment, DebruijnIndex, DomainGoal, GenericArg, GenericArgData, Goal, GoalData, Guidance, InEnvironment,
InferenceVar, Interner, Lifetime, OpaqueTyId, ParamKind, ProjectionTy, ProjectionTyExt, Scalar, InferenceVar, Interner, Lifetime, OpaqueTyId, ParamKind, ProjectionTy, ProjectionTyExt, Scalar,
Solution, Substitution, TraitEnvironment, Ty, TyBuilder, TyExt, TyKind, VariableKind, Solution, Substitution, TraitEnvironment, TraitRef, Ty, TyBuilder, TyExt, TyKind, VariableKind,
WhereClause, WhereClause,
}; };
@ -898,6 +899,33 @@ impl<'a> InferenceTable<'a> {
_ => c, _ => c,
} }
} }
/// Check if given type is `Sized` or not
pub(crate) fn is_sized(&mut self, ty: &Ty) -> bool {
if let Some((AdtId::StructId(id), subst)) = ty.as_adt() {
let struct_data = self.db.struct_data(id);
if let Some((last_field, _)) = struct_data.variant_data.fields().iter().last() {
let last_field_ty =
self.db.field_types(id.into())[last_field].clone().substitute(Interner, subst);
// Structs can have DST as its last field and such cases are not handled
// as unsized by the chalk, so we do this manually
return self.is_sized(&last_field_ty);
}
}
let Some(sized) = self
.db
.lang_item(self.trait_env.krate, LangItem::Sized)
.and_then(|sized| sized.as_trait())
else {
return false;
};
let sized_pred = WhereClause::Implemented(TraitRef {
trait_id: to_chalk_trait_id(sized),
substitution: Substitution::from1(Interner, ty.clone()),
});
let goal = GoalData::DomainGoal(chalk_ir::DomainGoal::Holds(sized_pred)).intern(Interner);
matches!(self.try_obligation(goal), Some(Solution::Unique(_)))
}
} }
impl fmt::Debug for InferenceTable<'_> { impl fmt::Debug for InferenceTable<'_> {

View file

@ -83,6 +83,7 @@ pub use autoderef::autoderef;
pub use builder::{ParamKind, TyBuilder}; pub use builder::{ParamKind, TyBuilder};
pub use chalk_ext::*; pub use chalk_ext::*;
pub use infer::{ pub use infer::{
cast::CastError,
closure::{CaptureKind, CapturedItem}, closure::{CaptureKind, CapturedItem},
could_coerce, could_unify, could_unify_deeply, Adjust, Adjustment, AutoBorrow, BindingMode, could_coerce, could_unify, could_unify_deeply, Adjust, Adjustment, AutoBorrow, BindingMode,
InferenceDiagnostic, InferenceResult, OverloadedDeref, PointerCast, InferenceDiagnostic, InferenceResult, OverloadedDeref, PointerCast,

View file

@ -837,7 +837,9 @@ pub enum CastKind {
PointerFromExposedAddress, PointerFromExposedAddress,
/// All sorts of pointer-to-pointer casts. Note that reference-to-raw-ptr casts are /// All sorts of pointer-to-pointer casts. Note that reference-to-raw-ptr casts are
/// translated into `&raw mut/const *r`, i.e., they are not actually casts. /// translated into `&raw mut/const *r`, i.e., they are not actually casts.
Pointer(PointerCast), PtrToPtr,
/// Pointer related casts that are done by coercions.
PointerCoercion(PointerCast),
/// Cast into a dyn* object. /// Cast into a dyn* object.
DynStar, DynStar,
IntToInt, IntToInt,

View file

@ -1475,7 +1475,7 @@ impl Evaluator<'_> {
} }
} }
Rvalue::Cast(kind, operand, target_ty) => match kind { Rvalue::Cast(kind, operand, target_ty) => match kind {
CastKind::Pointer(cast) => match cast { CastKind::PointerCoercion(cast) => match cast {
PointerCast::ReifyFnPointer | PointerCast::ClosureFnPointer(_) => { PointerCast::ReifyFnPointer | PointerCast::ClosureFnPointer(_) => {
let current_ty = self.operand_ty(operand, locals)?; let current_ty = self.operand_ty(operand, locals)?;
if let TyKind::FnDef(_, _) | TyKind::Closure(_, _) = if let TyKind::FnDef(_, _) | TyKind::Closure(_, _) =
@ -1506,6 +1506,7 @@ impl Evaluator<'_> {
}, },
CastKind::DynStar => not_supported!("dyn star cast"), CastKind::DynStar => not_supported!("dyn star cast"),
CastKind::IntToInt CastKind::IntToInt
| CastKind::PtrToPtr
| CastKind::PointerExposeAddress | CastKind::PointerExposeAddress
| CastKind::PointerFromExposedAddress => { | CastKind::PointerFromExposedAddress => {
let current_ty = self.operand_ty(operand, locals)?; let current_ty = self.operand_ty(operand, locals)?;

View file

@ -399,7 +399,7 @@ extern "C" {
fn memcmp(s1: *const u8, s2: *const u8, n: usize) -> i32; fn memcmp(s1: *const u8, s2: *const u8, n: usize) -> i32;
} }
fn my_cmp(x: &[u8], y: &[u8]) -> i32 { fn my_cmp(x: &[u8; 3], y: &[u8; 3]) -> i32 {
memcmp(x as *const u8, y as *const u8, x.len()) memcmp(x as *const u8, y as *const u8, x.len())
} }
@ -779,6 +779,7 @@ fn main() {
fn posix_getenv() { fn posix_getenv() {
check_pass( check_pass(
r#" r#"
//- minicore: sized
//- /main.rs env:foo=bar //- /main.rs env:foo=bar
type c_char = u8; type c_char = u8;

View file

@ -31,7 +31,7 @@ use crate::{
display::HirDisplay, display::HirDisplay,
error_lifetime, error_lifetime,
generics::generics, generics::generics,
infer::{CaptureKind, CapturedItem, TypeMismatch}, infer::{cast::CastTy, unify::InferenceTable, CaptureKind, CapturedItem, TypeMismatch},
inhabitedness::is_ty_uninhabited_from, inhabitedness::is_ty_uninhabited_from,
layout::LayoutError, layout::LayoutError,
mapping::ToChalk, mapping::ToChalk,
@ -362,7 +362,7 @@ impl<'ctx> MirLowerCtx<'ctx> {
current, current,
place, place,
Rvalue::Cast( Rvalue::Cast(
CastKind::Pointer(*cast), CastKind::PointerCoercion(*cast),
Operand::Copy(p), Operand::Copy(p),
last.target.clone(), last.target.clone(),
), ),
@ -898,14 +898,26 @@ impl<'ctx> MirLowerCtx<'ctx> {
let Some((it, current)) = self.lower_expr_to_some_operand(*expr, current)? else { let Some((it, current)) = self.lower_expr_to_some_operand(*expr, current)? else {
return Ok(None); return Ok(None);
}; };
let source_ty = self.infer[*expr].clone(); // Since we don't have THIR, this is the "zipped" version of [rustc's HIR lowering](https://github.com/rust-lang/rust/blob/e71f9529121ca8f687e4b725e3c9adc3f1ebab4d/compiler/rustc_mir_build/src/thir/cx/expr.rs#L165-L178)
let target_ty = self.infer[expr_id].clone(); // and [THIR lowering as RValue](https://github.com/rust-lang/rust/blob/a4601859ae3875732797873612d424976d9e3dd0/compiler/rustc_mir_build/src/build/expr/as_rvalue.rs#L193-L313)
self.push_assignment( let rvalue = if self.infer.coercion_casts.contains(expr) {
current, Rvalue::Use(it)
place, } else {
Rvalue::Cast(cast_kind(&source_ty, &target_ty)?, it, target_ty), let source_ty = self.infer[*expr].clone();
expr_id.into(), let target_ty = self.infer[expr_id].clone();
); let cast_kind = if source_ty.as_reference().is_some() {
CastKind::PointerCoercion(PointerCast::ArrayToPointer)
} else {
let mut table = InferenceTable::new(
self.db,
self.db.trait_environment_for_body(self.owner),
);
cast_kind(&mut table, &source_ty, &target_ty)?
};
Rvalue::Cast(cast_kind, it, target_ty)
};
self.push_assignment(current, place, rvalue, expr_id.into());
Ok(Some(current)) Ok(Some(current))
} }
Expr::Ref { expr, rawness: _, mutability } => { Expr::Ref { expr, rawness: _, mutability } => {
@ -2005,40 +2017,21 @@ impl<'ctx> MirLowerCtx<'ctx> {
} }
} }
fn cast_kind(source_ty: &Ty, target_ty: &Ty) -> Result<CastKind> { fn cast_kind(table: &mut InferenceTable<'_>, source_ty: &Ty, target_ty: &Ty) -> Result<CastKind> {
Ok(match (source_ty.kind(Interner), target_ty.kind(Interner)) { let from = CastTy::from_ty(table, source_ty);
(TyKind::FnDef(..), TyKind::Function(_)) => CastKind::Pointer(PointerCast::ReifyFnPointer), let cast = CastTy::from_ty(table, target_ty);
(TyKind::Scalar(s), TyKind::Scalar(t)) => match (s, t) { Ok(match (from, cast) {
(chalk_ir::Scalar::Float(_), chalk_ir::Scalar::Float(_)) => CastKind::FloatToFloat, (Some(CastTy::Ptr(..) | CastTy::FnPtr), Some(CastTy::Int(_))) => {
(chalk_ir::Scalar::Float(_), _) => CastKind::FloatToInt, CastKind::PointerExposeAddress
(_, chalk_ir::Scalar::Float(_)) => CastKind::IntToFloat,
(_, _) => CastKind::IntToInt,
},
(TyKind::Scalar(_), TyKind::Raw(..)) => CastKind::PointerFromExposedAddress,
(TyKind::Raw(..), TyKind::Scalar(_)) => CastKind::PointerExposeAddress,
(TyKind::Raw(_, a) | TyKind::Ref(_, _, a), TyKind::Raw(_, b) | TyKind::Ref(_, _, b)) => {
CastKind::Pointer(if a == b {
PointerCast::MutToConstPointer
} else if matches!(b.kind(Interner), TyKind::Slice(_))
&& matches!(a.kind(Interner), TyKind::Array(_, _))
|| matches!(b.kind(Interner), TyKind::Dyn(_))
{
PointerCast::Unsize
} else if matches!(a.kind(Interner), TyKind::Slice(s) if s == b) {
PointerCast::ArrayToPointer
} else {
// cast between two sized pointer, like *const i32 to *const i8, or two unsized pointer, like
// slice to slice, slice to str, ... . These are no-ops (even in the unsized case, no metadata
// will be touched) but there is no specific variant
// for it in `PointerCast` so we use `MutToConstPointer`
PointerCast::MutToConstPointer
})
} }
// Enum to int casts (Some(CastTy::Int(_)), Some(CastTy::Ptr(..))) => CastKind::PointerFromExposedAddress,
(TyKind::Scalar(_), TyKind::Adt(..)) | (TyKind::Adt(..), TyKind::Scalar(_)) => { (Some(CastTy::Int(_)), Some(CastTy::Int(_))) => CastKind::IntToInt,
CastKind::IntToInt (Some(CastTy::FnPtr), Some(CastTy::Ptr(..))) => CastKind::FnPtrToPtr,
} (Some(CastTy::Float), Some(CastTy::Int(_))) => CastKind::FloatToInt,
(a, b) => not_supported!("Unknown cast between {a:?} and {b:?}"), (Some(CastTy::Int(_)), Some(CastTy::Float)) => CastKind::IntToFloat,
(Some(CastTy::Float), Some(CastTy::Float)) => CastKind::FloatToFloat,
(Some(CastTy::Ptr(..)), Some(CastTy::Ptr(..))) => CastKind::PtrToPtr,
_ => not_supported!("Unknown cast between {source_ty:?} and {target_ty:?}"),
}) })
} }

View file

@ -1286,6 +1286,7 @@ fn main() {
fn method_on_dyn_impl() { fn method_on_dyn_impl() {
check_types( check_types(
r#" r#"
//- minicore: coerce_unsized
trait Foo {} trait Foo {}
impl Foo for u32 {} impl Foo for u32 {}

View file

@ -1907,6 +1907,7 @@ fn dont_unify_on_casts() {
// #15246 // #15246
check_types( check_types(
r#" r#"
//- minicore: sized
fn unify(_: [bool; 1]) {} fn unify(_: [bool; 1]) {}
fn casted(_: *const bool) {} fn casted(_: *const bool) {}
fn default<T>() -> T { loop {} } fn default<T>() -> T { loop {} }
@ -1926,6 +1927,7 @@ fn test() {
fn rustc_test_issue_52437() { fn rustc_test_issue_52437() {
check_types( check_types(
r#" r#"
//- minicore: sized
fn main() { fn main() {
let x = [(); &(&'static: loop { |x| {}; }) as *const _ as usize] let x = [(); &(&'static: loop { |x| {}; }) as *const _ as usize]
//^ [(); _] //^ [(); _]

View file

@ -3572,6 +3572,7 @@ fn f<T>(t: Ark<T>) {
fn ref_to_array_to_ptr_cast() { fn ref_to_array_to_ptr_cast() {
check_types( check_types(
r#" r#"
//- minicore: sized
fn default<T>() -> T { loop {} } fn default<T>() -> T { loop {} }
fn foo() { fn foo() {
let arr = [default()]; let arr = [default()];

View file

@ -4,7 +4,9 @@
//! This probably isn't the best way to do this -- ideally, diagnostics should //! This probably isn't the best way to do this -- ideally, diagnostics should
//! be expressed in terms of hir types themselves. //! be expressed in terms of hir types themselves.
pub use hir_ty::diagnostics::{CaseType, IncorrectCase}; pub use hir_ty::diagnostics::{CaseType, IncorrectCase};
use hir_ty::{db::HirDatabase, diagnostics::BodyValidationDiagnostic, InferenceDiagnostic}; use hir_ty::{
db::HirDatabase, diagnostics::BodyValidationDiagnostic, CastError, InferenceDiagnostic,
};
use cfg::{CfgExpr, CfgOptions}; use cfg::{CfgExpr, CfgOptions};
use either::Either; use either::Either;
@ -50,10 +52,12 @@ macro_rules! diagnostics {
diagnostics![ diagnostics![
AwaitOutsideOfAsync, AwaitOutsideOfAsync,
BreakOutsideOfLoop, BreakOutsideOfLoop,
CastToUnsized,
ExpectedFunction, ExpectedFunction,
InactiveCode, InactiveCode,
IncoherentImpl, IncoherentImpl,
IncorrectCase, IncorrectCase,
InvalidCast,
InvalidDeriveTarget, InvalidDeriveTarget,
MacroDefError, MacroDefError,
MacroError, MacroError,
@ -364,6 +368,20 @@ pub struct RemoveUnnecessaryElse {
pub if_expr: InFile<AstPtr<ast::IfExpr>>, pub if_expr: InFile<AstPtr<ast::IfExpr>>,
} }
#[derive(Debug)]
pub struct CastToUnsized {
pub expr: InFile<AstPtr<ast::Expr>>,
pub cast_ty: Type,
}
#[derive(Debug)]
pub struct InvalidCast {
pub expr: InFile<AstPtr<ast::Expr>>,
pub error: CastError,
pub expr_ty: Type,
pub cast_ty: Type,
}
impl AnyDiagnostic { impl AnyDiagnostic {
pub(crate) fn body_validation_diagnostic( pub(crate) fn body_validation_diagnostic(
db: &dyn HirDatabase, db: &dyn HirDatabase,
@ -620,6 +638,16 @@ impl AnyDiagnostic {
}; };
MismatchedTupleStructPatArgCount { expr_or_pat, expected, found }.into() MismatchedTupleStructPatArgCount { expr_or_pat, expected, found }.into()
} }
InferenceDiagnostic::CastToUnsized { expr, cast_ty } => {
let expr = expr_syntax(*expr)?;
CastToUnsized { expr, cast_ty: Type::new(db, def, cast_ty.clone()) }.into()
}
InferenceDiagnostic::InvalidCast { expr, error, expr_ty, cast_ty } => {
let expr = expr_syntax(*expr)?;
let expr_ty = Type::new(db, def, expr_ty.clone());
let cast_ty = Type::new(db, def, cast_ty.clone());
InvalidCast { expr, error: *error, expr_ty, cast_ty }.into()
}
}) })
} }
} }

View file

@ -147,7 +147,7 @@ pub use {
layout::LayoutError, layout::LayoutError,
mir::{MirEvalError, MirLowerError}, mir::{MirEvalError, MirLowerError},
object_safety::{MethodViolationCode, ObjectSafetyViolation}, object_safety::{MethodViolationCode, ObjectSafetyViolation},
FnAbi, PointerCast, Safety, CastError, FnAbi, PointerCast, Safety,
}, },
// FIXME: Properly encapsulate mir // FIXME: Properly encapsulate mir
hir_ty::{mir, Interner as ChalkTyInterner}, hir_ty::{mir, Interner as ChalkTyInterner},

View file

@ -1020,6 +1020,7 @@ fn main() {
check_assist( check_assist(
inline_call, inline_call,
r#" r#"
//- minicore: sized
fn foo(x: *const u32) -> u32 { fn foo(x: *const u32) -> u32 {
x as u32 x as u32
} }

View file

@ -333,7 +333,8 @@ fn foo() {
check_assist( check_assist(
inline_local_variable, inline_local_variable,
r" r"
fn bar(a: usize): usize { a } //- minicore: sized
fn bar(a: usize) -> usize { a }
fn foo() { fn foo() {
let a$0 = bar(1) as u64; let a$0 = bar(1) as u64;
a + 1; a + 1;
@ -347,7 +348,7 @@ fn foo() {
bar(a); bar(a);
}", }",
r" r"
fn bar(a: usize): usize { a } fn bar(a: usize) -> usize { a }
fn foo() { fn foo() {
(bar(1) as u64) + 1; (bar(1) as u64) + 1;
if (bar(1) as u64) > 10 { if (bar(1) as u64) > 10 {

File diff suppressed because it is too large Load diff

View file

@ -99,8 +99,9 @@ mod tests {
fn missing_unsafe_diagnostic_with_raw_ptr() { fn missing_unsafe_diagnostic_with_raw_ptr() {
check_diagnostics( check_diagnostics(
r#" r#"
//- minicore: sized
fn main() { fn main() {
let x = &5 as *const usize; let x = &5_usize as *const usize;
unsafe { let _y = *x; } unsafe { let _y = *x; }
let _z = *x; let _z = *x;
} //^^💡 error: this operation is unsafe and requires an unsafe function or block } //^^💡 error: this operation is unsafe and requires an unsafe function or block
@ -112,17 +113,18 @@ fn main() {
fn missing_unsafe_diagnostic_with_unsafe_call() { fn missing_unsafe_diagnostic_with_unsafe_call() {
check_diagnostics( check_diagnostics(
r#" r#"
//- minicore: sized
struct HasUnsafe; struct HasUnsafe;
impl HasUnsafe { impl HasUnsafe {
unsafe fn unsafe_fn(&self) { unsafe fn unsafe_fn(&self) {
let x = &5 as *const usize; let x = &5_usize as *const usize;
let _y = *x; let _y = *x;
} }
} }
unsafe fn unsafe_fn() { unsafe fn unsafe_fn() {
let x = &5 as *const usize; let x = &5_usize as *const usize;
let _y = *x; let _y = *x;
} }
@ -250,14 +252,15 @@ fn main() {
fn add_unsafe_block_when_dereferencing_a_raw_pointer() { fn add_unsafe_block_when_dereferencing_a_raw_pointer() {
check_fix( check_fix(
r#" r#"
//- minicore: sized
fn main() { fn main() {
let x = &5 as *const usize; let x = &5_usize as *const usize;
let _z = *x$0; let _z = *x$0;
} }
"#, "#,
r#" r#"
fn main() { fn main() {
let x = &5 as *const usize; let x = &5_usize as *const usize;
let _z = unsafe { *x }; let _z = unsafe { *x };
} }
"#, "#,
@ -268,8 +271,9 @@ fn main() {
fn add_unsafe_block_when_calling_unsafe_function() { fn add_unsafe_block_when_calling_unsafe_function() {
check_fix( check_fix(
r#" r#"
//- minicore: sized
unsafe fn func() { unsafe fn func() {
let x = &5 as *const usize; let x = &5_usize as *const usize;
let z = *x; let z = *x;
} }
fn main() { fn main() {
@ -278,7 +282,7 @@ fn main() {
"#, "#,
r#" r#"
unsafe fn func() { unsafe fn func() {
let x = &5 as *const usize; let x = &5_usize as *const usize;
let z = *x; let z = *x;
} }
fn main() { fn main() {
@ -292,6 +296,7 @@ fn main() {
fn add_unsafe_block_when_calling_unsafe_method() { fn add_unsafe_block_when_calling_unsafe_method() {
check_fix( check_fix(
r#" r#"
//- minicore: sized
struct S(usize); struct S(usize);
impl S { impl S {
unsafe fn func(&self) { unsafe fn func(&self) {

View file

@ -190,4 +190,16 @@ fn foo(mut slice: &[u32]) -> usize {
"#, "#,
); );
} }
#[test]
fn regression_16564() {
check_diagnostics(
r#"
//- minicore: copy
fn test() {
let _x = (&(&mut (),)).0 as *const ();
}
"#,
)
}
} }

View file

@ -30,6 +30,7 @@ mod handlers {
pub(crate) mod inactive_code; pub(crate) mod inactive_code;
pub(crate) mod incoherent_impl; pub(crate) mod incoherent_impl;
pub(crate) mod incorrect_case; pub(crate) mod incorrect_case;
pub(crate) mod invalid_cast;
pub(crate) mod invalid_derive_target; pub(crate) mod invalid_derive_target;
pub(crate) mod macro_error; pub(crate) mod macro_error;
pub(crate) mod malformed_derive; pub(crate) mod malformed_derive;
@ -390,6 +391,7 @@ pub fn semantic_diagnostics(
for diag in diags { for diag in diags {
let d = match diag { let d = match diag {
AnyDiagnostic::AwaitOutsideOfAsync(d) => handlers::await_outside_of_async::await_outside_of_async(&ctx, &d), AnyDiagnostic::AwaitOutsideOfAsync(d) => handlers::await_outside_of_async::await_outside_of_async(&ctx, &d),
AnyDiagnostic::CastToUnsized(d) => handlers::invalid_cast::cast_to_unsized(&ctx, &d),
AnyDiagnostic::ExpectedFunction(d) => handlers::expected_function::expected_function(&ctx, &d), AnyDiagnostic::ExpectedFunction(d) => handlers::expected_function::expected_function(&ctx, &d),
AnyDiagnostic::InactiveCode(d) => match handlers::inactive_code::inactive_code(&ctx, &d) { AnyDiagnostic::InactiveCode(d) => match handlers::inactive_code::inactive_code(&ctx, &d) {
Some(it) => it, Some(it) => it,
@ -397,6 +399,7 @@ pub fn semantic_diagnostics(
} }
AnyDiagnostic::IncoherentImpl(d) => handlers::incoherent_impl::incoherent_impl(&ctx, &d), AnyDiagnostic::IncoherentImpl(d) => handlers::incoherent_impl::incoherent_impl(&ctx, &d),
AnyDiagnostic::IncorrectCase(d) => handlers::incorrect_case::incorrect_case(&ctx, &d), AnyDiagnostic::IncorrectCase(d) => handlers::incorrect_case::incorrect_case(&ctx, &d),
AnyDiagnostic::InvalidCast(d) => handlers::invalid_cast::invalid_cast(&ctx, &d),
AnyDiagnostic::InvalidDeriveTarget(d) => handlers::invalid_derive_target::invalid_derive_target(&ctx, &d), AnyDiagnostic::InvalidDeriveTarget(d) => handlers::invalid_derive_target::invalid_derive_target(&ctx, &d),
AnyDiagnostic::MacroDefError(d) => handlers::macro_error::macro_def_error(&ctx, &d), AnyDiagnostic::MacroDefError(d) => handlers::macro_error::macro_def_error(&ctx, &d),
AnyDiagnostic::MacroError(d) => handlers::macro_error::macro_error(&ctx, &d), AnyDiagnostic::MacroError(d) => handlers::macro_error::macro_error(&ctx, &d),

View file

@ -313,6 +313,7 @@ fn main() {
//^^^^^^^^^^^^<safe-fn-pointer-to-unsafe-fn-pointer> //^^^^^^^^^^^^<safe-fn-pointer-to-unsafe-fn-pointer>
//^^^^^^^^^^^^( //^^^^^^^^^^^^(
//^^^^^^^^^^^^) //^^^^^^^^^^^^)
//^^^^<fn-item-to-fn-pointer>
let _: fn() = || {}; let _: fn() = || {};
//^^^^^<closure-to-fn-pointer> //^^^^^<closure-to-fn-pointer>
let _: unsafe fn() = || {}; let _: unsafe fn() = || {};
@ -321,6 +322,8 @@ fn main() {
//^^^^^^^^^^^^^^^^^^^^^<mut-ptr-to-const-ptr> //^^^^^^^^^^^^^^^^^^^^^<mut-ptr-to-const-ptr>
//^^^^^^^^^^^^^^^^^^^^^( //^^^^^^^^^^^^^^^^^^^^^(
//^^^^^^^^^^^^^^^^^^^^^) //^^^^^^^^^^^^^^^^^^^^^)
//^^^^^^^^^&raw mut $
//^^^^^^^^^*
let _: &mut [_] = &mut [0; 0]; let _: &mut [_] = &mut [0; 0];
//^^^^^^^^^^^<unsize> //^^^^^^^^^^^<unsize>
//^^^^^^^^^^^&mut $ //^^^^^^^^^^^&mut $

View file

@ -554,6 +554,7 @@ fn main() {
fn test_unsafe_highlighting() { fn test_unsafe_highlighting() {
check_highlighting( check_highlighting(
r#" r#"
//- minicore: sized
macro_rules! id { macro_rules! id {
($($tt:tt)*) => { ($($tt:tt)*) => {
$($tt)* $($tt)*